You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by squito <gi...@git.apache.org> on 2016/07/06 22:26:02 UTC

[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

GitHub user squito opened a pull request:

    https://github.com/apache/spark/pull/14079

    [SPARK-8425][CORE] New Blacklist Mechanism

    ## What changes were proposed in this pull request?
    
    This adds a blacklisting mechanism for Spark to handle bad executors and nodes, eg. for when a disk fails.  Previously there was an undocumented blacklisting mechanism, but it was very limited.  This supports blacklisting individual tasks on an executor, executors and nodes for an entire stage, and executors and nodes across an entire spark application.  Full details are in a design doc attached to the jira.
    
    ## How was this patch tested?
    
    Added unit tests, ran them via Jenkins, also ran a handful of them in a loop to check for flakiness.
    
    The added tests include:
    * verifying BlacklistTracker works correctly
    * verifying TaskSchedulerImpl interacts with BlacklistTracker correctly (via a mock BlacklistTracker)
    * an integration test for the entire scheduler with blacklisting in a few different scenarios
    
    TODO
    [ ] More testing on a real cluster
    [ ] Fix & test handling of shuffle-fetch failures
    [ ] Documentation

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/squito/spark blacklist-SPARK-8425

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/14079.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #14079
    
----
commit 9a6aaede723b206a95a616ae740dcc5ed6b74cbc
Author: mwws <we...@intel.com>
Date:   2015-12-29T06:01:17Z

    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.

commit 5bfe94106f8c49f2c3d60d13bd2bb6389be94e4f
Author: Imran Rashid <ir...@cloudera.com>
Date:   2016-05-10T17:49:05Z

    Update for new design

----


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/62352/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #67271 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/67271/consoleFull)** for PR 14079 at commit [`162cb0d`](https://github.com/apache/spark/commit/162cb0d4253cddddacf93d7e6985c01d730ec27e).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r72525188
  
    --- 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")
    +        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 (executorIdToBlacklistStatus.nonEmpty) {
    +        nextExpiryTime = executorIdToBlacklistStatus.map{_._2.expiryTime}.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(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
    +        executorIdToBlacklistStatus.put(exec, BlacklistedExecutor(node, expiryTime))
    +        val blacklistedExecsOnNode = nodeToFailedExecs.getOrElseUpdate(node, HashSet[String]())
    +        blacklistedExecsOnNode += exec
    +        executorIdToFailureCount.remove(exec)
    +        if (expiryTime < nextExpiryTime) {
    +          nextExpiryTime = expiryTime
    +        }
    +
    +        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)
    +      }
    +    }
    +  }
    +
    +  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 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.
    +    executorIdToFailureCount -= executorId
    +  }
    +}
    +
    +
    +private[scheduler] object BlacklistTracker extends Logging {
    +
    +  /**
    +   * 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 = {
    +    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 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) =>
    +              // 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)
    +        }
    +    }
    +  }
    +
    +  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")
    --- End diff --
    
    make 1h a constant in the object, so it doesn't get lost?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r72487636
  
    --- 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
    --- End diff --
    
    super nit but can you write this as a bulleted list so it's easy to skim?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #64498 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/64498/consoleFull)** for PR 14079 at commit [`cc3b968`](https://github.com/apache/spark/commit/cc3b9683940eebc930e9bf7a5088dc987da23595).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r72536190
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -310,12 +343,41 @@ 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.foreach(_.expireExecutorsInBlacklist())
    +
    +    val sortedTaskSets = rootPool.getSortedTaskSetQueue
    +    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
    +      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()) {
    --- End diff --
    
    This is an optimization right? (it's not needed for correctness?)  If so, can we remove it? Worried about the complexity of all of this -- and this seems rare enough that the delta in time to failure isn't super important.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/63993/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #64021 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/64021/consoleFull)** for PR 14079 at commit [`8c58ad9`](https://github.com/apache/spark/commit/8c58ad96db948bec95f2e99fb9298db481e60b58).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r72538697
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala ---
    @@ -770,9 +794,19 @@ 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())
    +
    +    // 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
    --- End diff --
    
    the code on the old line 710 makes it seem like we always expect this to be a TaskFailedReason (should the param type just be changed?)?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r72501035
  
    --- 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.
    --- End diff --
    
    Mapping from nodes to all of the executors that have been blacklisted on that node. We do *not* ...


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r86600764
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -306,12 +323,24 @@ private[spark] class TaskSchedulerImpl(
           }
         }
     
    +    // 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 =>
    --- End diff --
    
    bl -> blacklistTracker (if you change blacklistTracker to be blacklistTrackerOpt)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/67286/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r71226362
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -0,0 +1,221 @@
    +/*
    + * 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
    +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_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] = 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(
    +      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.
    +    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())
    --- End diff --
    
    this is only getting the alive ones, what about dynamic allocation and if any executor leaves and then later another gets reallocated on that node?  I would actually expect this to happen if we blacklist an executor on that node, it won't get anything scheduled on it so I would think it would become idle and then get released.  If something gets reallocated on it, then would again get an executor failure but because we aren't tracking the one that was released we don't hit the node blacklisting setting.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r74326330
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -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: Option[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))
    --- End diff --
    
    I don't mind changing this, but I didn't think that was our style convention.  I thought that was only for declaring the parameters of methods -- not for their call sites.  Maybe its different for invocations of `this()`, but I even see counter-examples to that, eg. :
    https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/TaskEndReason.scala#L133
    https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala#L47


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r86611855
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala ---
    @@ -17,10 +17,299 @@
     
     package org.apache.spark.scheduler
     
    -import org.apache.spark.{SparkConf, SparkFunSuite}
    +import org.mockito.Mockito.when
    +import org.scalatest.BeforeAndAfterEach
    +import org.scalatest.mock.MockitoSugar
    +
    +import org.apache.spark._
     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
    +  }
     
    -class BlacklistTrackerSuite extends SparkFunSuite {
    +  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
    --- End diff --
    
    since the count of failures on that executor should have been reset to 0


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r77296679
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.BeforeAndAfterEach
    +import org.scalatest.mock.MockitoSugar
    +
    +import org.apache.spark._
    +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 blacklistTracker: BlacklistTracker = _
    +
    +  override def afterEach(): Unit = {
    +    if (blacklistTracker != null) {
    +      blacklistTracker = null
    +    }
    +    super.afterEach()
    +  }
    +
    +  val allOptions = (('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 = {
    +    allOptions.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
    +  }
    +
    +  test("Blacklisting individual tasks") {
    +    val conf = new SparkConf().setAppName("test").setMaster("local")
    +      .set(config.BLACKLIST_ENABLED.key, "true")
    +    val scheduler = mockTaskSchedWithConf(conf)
    +    // 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)
    +    tsm.updateBlacklistForFailedTask("hostA", "1", 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(blacklistTracker.nodeBlacklist() === Set())
    +    assertEquivalentToSet(blacklistTracker.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,
    +    // 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)
    +    // 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 badPart = (index == 0 || index == 1)
    +        val taskExp = (badExec && badPart)
    +        assert(
    +          tsm.isExecutorBlacklistedForTask(executor, index) === taskExp)
    +        val executorExp = badExec
    +        assert(tsm.isExecutorBlacklistedForTaskSet(executor) === executorExp)
    +      }
    +    }
    +    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())
    +
    +    // 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()
    +  }
    +
    +  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)
    +    blacklistTracker = new BlacklistTracker(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 =>
    +      val taskSet = FakeTask.createTaskSet(1)
    +      val tsm = new TaskSetManager(scheduler, Some(tracker), taskSet, 4, clock)
    +      tsm.updateBlacklistForFailedTask("hostA", "1", 0)
    +      tracker.updateBlacklistForSuccessfulTaskSet(stage, 0, tsm.execToFailures)
    +    }
    +    assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1"))
    --- End diff --
    
    like above,  can you add incremental assertions in the for-loop to make sure that the executor is *not* blacklisted until enough things fail?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r77296312
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.BeforeAndAfterEach
    +import org.scalatest.mock.MockitoSugar
    +
    +import org.apache.spark._
    +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 blacklistTracker: BlacklistTracker = _
    +
    +  override def afterEach(): Unit = {
    +    if (blacklistTracker != null) {
    +      blacklistTracker = null
    +    }
    +    super.afterEach()
    +  }
    +
    +  val allOptions = (('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 = {
    +    allOptions.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
    +  }
    +
    +  test("Blacklisting individual tasks") {
    +    val conf = new SparkConf().setAppName("test").setMaster("local")
    +      .set(config.BLACKLIST_ENABLED.key, "true")
    +    val scheduler = mockTaskSchedWithConf(conf)
    +    // 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)
    +    tsm.updateBlacklistForFailedTask("hostA", "1", 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(blacklistTracker.nodeBlacklist() === Set())
    +    assertEquivalentToSet(blacklistTracker.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,
    +    // 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)
    +    // 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 badPart = (index == 0 || index == 1)
    +        val taskExp = (badExec && badPart)
    +        assert(
    +          tsm.isExecutorBlacklistedForTask(executor, index) === taskExp)
    +        val executorExp = badExec
    +        assert(tsm.isExecutorBlacklistedForTaskSet(executor) === executorExp)
    +      }
    +    }
    +    assertEquivalentToSet(tsm.isNodeBlacklistedForTaskSet, Set("hostA"))
    --- End diff --
    
    can you also add checks above (in between each updateBlacklist... call) that make sure hostA hasn't yet been 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r72501415
  
    --- 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
    --- End diff --
    
    execsToRevive? execsToUnblacklist?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/64499/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #69428 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/69428/consoleFull)** for PR 14079 at commit [`35978e2`](https://github.com/apache/spark/commit/35978e28c78ddac67613d01f6653cf8b3451b42a).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r70725583
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -0,0 +1,223 @@
    +/*
    + * 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
    +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_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] = 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) }
    --- End diff --
    
    minor: not sure how much you're worried about performance here, but `execsToClear.retain` might be a little faster than filter + remove, at the expense of losing the log message.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r88088492
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -17,10 +17,254 @@
     
     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.Utils
    +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 = new  HashMap[String, ExecutorFailureList]()
    +  val executorIdToBlacklistStatus = new HashMap[String, BlacklistedExecutor]()
    +  val nodeIdToBlacklistExpiryTime = new HashMap[String, Long]()
    +  /**
    +   * An immutable copy of the set of nodes that are currently blacklisted.  Kept in an
    +   * AtomicReference to make [[nodeBlacklist()]] thread-safe.
    +   */
    +  private val _nodeBlacklist = new AtomicReference[Set[String]](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 nodeToBlacklistedExecs = new HashMap[String, HashSet[String]]()
    +
    +  /**
    +   * Un-blacklists executors and nodes that have been blacklisted for at least
    +   * BLACKLIST_TIMEOUT_MILLIS
    +   */
    +  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"for those executors has timed out")
    +        execsToUnblacklist.foreach { exec =>
    +          val status = executorIdToBlacklistStatus.remove(exec).get
    +          val failedExecsOnNode = nodeToBlacklistedExecs(status.node)
    +          failedExecsOnNode.remove(exec)
    +          if (failedExecsOnNode.isEmpty) {
    +            nodeToBlacklistedExecs.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 = {
    +    // we don't need to check nodeIdToBlacklistExpiryTime because that will always share an
    +    // expiry time with some blacklisted executor.  In other words, the next node expiry time
    +    // will never be before the next executor blacklist time.
    +    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()
    --- End diff --
    
    well, I think it depends what you mean by "handled correctly".  We use the time the taskset completes, so its OK if the failures happened long ago when the taskset started, we still count those failures in the app blacklist, so later failures can trickle in and push us over the limit.
    
    OTOH, this also means that if we were already close to the limit on failures for the application when this taskset started, then a really long running taskset will fail to push us over the limit -- by the time the latest task set finishes, we've expired the old failures, so we only get failures from the new taskset.  So if your taskset time is longer than the blacklist timeout, you're unlikely to ever get application level blacklisting.
    
    Clearly this is not great, but its not *that* bad.  After all, even if it were app-level blacklisted, we'd hit still the timeout and remove the bad resources from the blacklist, so that we'd need to rediscover it in future blacklists.  One of the main reasons for the app-level blacklist is to avoid lots of failures when the tasksets are *short*.  If you really want an application level blacklist which is useful across really long tasksets, then you've got to crank up your timeout.
    
    We could change this slightly by *first* updating the application level blacklist, and *then* expiring failures past the timeout.  But to me that behavior seems much less intuitive, for a pretty questionable gain.
    
    Does that make sense?  What do you think?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r70519430
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -0,0 +1,330 @@
    +/*
    + * 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
    +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
    --- End diff --
    
    nit spacing off.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r86649370
  
    --- Diff: yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala ---
    @@ -121,13 +121,20 @@ private[spark] abstract class YarnSchedulerBackend(
         }
       }
     
    +  private[cluster] def prepareRequestExecutors(requestedTotal: Int): RequestExecutors = {
    --- End diff --
    
    do you need a new method for this? (can it just be part of doRequestTotalExecutors?) (See comment in test)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r70004226
  
    --- 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
    +   * the executor is blacklisted overall (or anything to do with the node the executor is on).
    +   */
    +  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)
    +  }
    +
    +  def nodeBlacklist(): Set[String] = {
    +    _nodeBlacklist.get()
    +  }
    +
    +  def isNodeBlacklisted(node: String): Boolean = {
    +    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 previous
    +    // 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.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
    --- End diff --
    
    nit: "This", "Let's"


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #64242 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/64242/consoleFull)** for PR 14079 at commit [`e56bb90`](https://github.com/apache/spark/commit/e56bb90152b4198556773f5cbc39a192107eeae2).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/63990/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by jsoltren <gi...@git.apache.org>.
Github user jsoltren commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r75392592
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala ---
    @@ -809,32 +821,65 @@ 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",
             "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 === 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 =>
    --- End diff --
    
    I worry about another hardcoded policy here. Fine for now, perhaps something to revisit later.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #64721 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/64721/consoleFull)** for PR 14079 at commit [`1297788`](https://github.com/apache/spark/commit/1297788d323f2f7ae996e547168f3a70a8fd122e).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/67205/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r86648176
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala ---
    @@ -17,10 +17,299 @@
     
     package org.apache.spark.scheduler
     
    -import org.apache.spark.{SparkConf, SparkFunSuite}
    +import org.mockito.Mockito.when
    +import org.scalatest.BeforeAndAfterEach
    +import org.scalatest.mock.MockitoSugar
    +
    +import org.apache.spark._
     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
    +  }
     
    -class BlacklistTrackerSuite extends SparkFunSuite {
    +  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.nodeToBlacklistedExecs("hostA").contains("1"))
    +    clock.advance(blacklist.BLACKLIST_TIMEOUT_MILLIS)
    +    blacklist.applyBlacklistTimeout()
    +    assert(!blacklist.nodeIdToBlacklistExpiryTime.contains("hostA"))
    +  }
    +
    +  test("task failures expire with time") {
    --- End diff --
    
    can you add an overall comment about what this does? like
    
    Verifies that 2 failures within the timeout period cause an executor to be blacklisted, but if task failures are spaced out by more than the timeout period, the first failure is timed out, and the executor isn't 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r76106200
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -0,0 +1,385 @@
    +/*
    + * 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.{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.
    +   */
    +  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.
    +   */
    +  private 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 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.
    +        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)
    +          }
    +        }
    +      }
    +      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.
    +        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 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 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" +
    +          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.
    +        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
    +  }
    +}
    +
    +
    +private[scheduler] object BlacklistTracker extends Logging {
    +
    +  private val DEFAULT_TIMEOUT = "1h"
    +
    +  /**
    +   * 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:
    +   *   - off for local mode
    +   *   - on for distributed modes (including local-cluster)
    +   */
    +  def isBlacklistEnabled(conf: SparkConf): Boolean = {
    +    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 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
    +            } 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
    +            !Utils.isLocalMaster(conf)
    +        }
    +    }
    +  }
    +
    +  def getBlacklistTimeout(conf: SparkConf): Long = {
    +    conf.get(config.BLACKLIST_TIMEOUT_CONF).getOrElse {
    +      conf.get(config.BLACKLIST_LEGACY_TIMEOUT_CONF).getOrElse {
    +        Utils.timeStringAsMs(DEFAULT_TIMEOUT)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * 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 */
    +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 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"
    +  }
    +}
    +
    +/**
    + * 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
    --- End diff --
    
    I'm seeing an exception on this when a node is unblacklisted after the timeout. In this case it blacklisted a set of executors and nodes and then after a 10 minutes timeout went to unblacklist.
    
    java.util.NoSuchElementException: next on empty iterator
            at scala.collection.Iterator$$anon$2.next(Iterator.scala:39)
            at scala.collection.Iterator$$anon$2.next(Iterator.scala:37)
            at scala.collection.IndexedSeqLike$Elements.next(IndexedSeqLike.scala:63)
            at scala.collection.IterableLike$class.head(IterableLike.scala:107)
            at scala.collection.mutable.ArrayBuffer.scala$collection$IndexedSeqOptimized$$super$head(ArrayBuffer.scala:48)
            at scala.collection.IndexedSeqOptimized$class.head(IndexedSeqOptimized.scala:126)
            at scala.collection.mutable.ArrayBuffer.head(ArrayBuffer.scala:48)
            at org.apache.spark.scheduler.ExecutorFailureList.minExpiryTime(BlacklistTracker.scala:305)
            at org.apache.spark.scheduler.ExecutorFailureList.dropFailuresWithTimeoutBefore(BlacklistTracker.scala:314)
            at org.apache.spark.scheduler.BlacklistTracker$$anonfun$applyBlacklistTimeout$1.apply(BlacklistTracker.scala:100)
            at org.apache.spark.scheduler.BlacklistTracker$$anonfun$applyBlacklistTimeout$1.apply(BlacklistTracker.scala:99)
            at scala.collection.mutable.HashMap$$anon$2$$anonfun$foreach$3.apply(HashMap.scala:108)
            at scala.collection.mutable.HashMap$$anon$2$$anonfun$foreach$3.apply(HashMap.scala:108)
            at scala.collection.mutable.HashTable$class.foreachEntry(HashTable.scala:230)
            at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:40)
            at scala.collection.mutable.HashMap$$anon$2.foreach(HashMap.scala:108)
            at org.apache.spark.scheduler.BlacklistTracker.applyBlacklistTimeout(BlacklistTracker.scala:99)
            at org.apache.spark.scheduler.TaskSchedulerImpl$$anonfun$resourceOffers$2.apply(TaskSchedulerImpl.scala:331)
            at org.apache.spark.scheduler.TaskSchedulerImpl$$anonfun$resourceOffers$2.apply(TaskSchedulerImpl.scala:331)
            at scala.Option.foreach(Option.scala:257)
            at org.apache.spark.scheduler.TaskSchedulerImpl.resourceOffers(TaskSchedulerImpl.scala:331)
            at org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend$DriverEndpoint.org$apache$spark$scheduler$cluster$CoarseGrainedSchedulerBackend$DriverEndpoint$$makeOffers(CoarseGrainedSchedulerBackend.scala:217)
            at org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend$DriverEndpoint$$anonfun$receive$1.applyOrElse(CoarseGrainedSchedulerBackend.scala:135)
            at org.apache.spark.rpc.netty.Inbox$$anonfun$process$1.apply$mcV$sp(Inbox.scala:117)
            at org.apache.spark.rpc.netty.Inbox.safelyCall(Inbox.scala:205)
            at org.apache.spark.rpc.netty.Inbox.process(Inbox.scala:101)
            at org.apache.spark.rpc.netty.Dispatcher$MessageLoop.run(Dispatcher.scala:213)
            at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
            at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
            at java.lang.Thread.run(Thread.java:745)



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r79934114
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.BeforeAndAfterEach
    +import org.scalatest.mock.MockitoSugar
    +
    +import org.apache.spark._
    +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 blacklistTracker: BlacklistTracker = _
    +
    +  override def afterEach(): Unit = {
    +    if (blacklistTracker != null) {
    +      blacklistTracker = null
    +    }
    +    super.afterEach()
    +  }
    +
    +  val allOptions = (('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 = {
    +    allOptions.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
    +  }
    +
    +  test("Blacklisting individual tasks") {
    --- End diff --
    
    yeah good point, I agree


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #67205 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/67205/consoleFull)** for PR 14079 at commit [`21907a5`](https://github.com/apache/spark/commit/21907a58d6e089940cf3a3a0f27b32460d43de2b).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r76683789
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -0,0 +1,395 @@
    +/*
    + * 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.{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[scheduler] 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.
    +   */
    +  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
    +   * 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
    +  }
    +}
    +
    +
    +private[scheduler] object BlacklistTracker extends Logging {
    +
    +  private val DEFAULT_TIMEOUT = "1h"
    +
    +  /**
    +   * 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:
    +   *   - off for local mode
    +   *   - on for distributed modes (including local-cluster)
    +   */
    +  def isBlacklistEnabled(conf: SparkConf): Boolean = {
    +    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 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
    +            } 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
    +            !Utils.isLocalMaster(conf)
    +        }
    +    }
    +  }
    +
    +  def getBlacklistTimeout(conf: SparkConf): Long = {
    +    conf.get(config.BLACKLIST_TIMEOUT_CONF).getOrElse {
    +      conf.get(config.BLACKLIST_LEGACY_TIMEOUT_CONF).getOrElse {
    +        Utils.timeStringAsMs(DEFAULT_TIMEOUT)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * 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 */
    +private[scheduler] final class ExecutorFailuresInTaskSet(val node: String) {
    --- End diff --
    
    I don't see parameter "node" is every used here


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r72487474
  
    --- 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
    --- End diff --
    
    nit: "It supports blacklisting executors for a specific task, blacklisting executors and nodes for all tasks in a stage, and blacklisting executors and nodes across an entire application (with a periodic expiry)."


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r71772380
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -0,0 +1,221 @@
    +/*
    + * 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
    +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_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] = 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(
    +      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.
    +    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())
    --- End diff --
    
    yeah we should just make sure not to count preempted against executor failures. There aren't any other reasons I can think of that we should ignore.  Hopefully countTowardsTaskFailures would be updated if there are.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r86611659
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala ---
    @@ -17,10 +17,299 @@
     
     package org.apache.spark.scheduler
     
    -import org.apache.spark.{SparkConf, SparkFunSuite}
    +import org.mockito.Mockito.when
    +import org.scalatest.BeforeAndAfterEach
    +import org.scalatest.mock.MockitoSugar
    +
    +import org.apache.spark._
     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
    +  }
     
    -class BlacklistTrackerSuite extends SparkFunSuite {
    +  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 =>
    --- End diff --
    
    same re:comment


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r92078684
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -678,4 +716,13 @@ private[spark] object TaskSchedulerImpl {
     
         retval.toList
       }
    +
    +  private def maybeCreateBlacklistTracker(conf: SparkConf): Option[BlacklistTracker] = {
    --- End diff --
    
    does it make sense for this method to be in the BlacklistTracker object? (then it could just be BlacklistTracker.maybeCreate()?)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r92286111
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala ---
    @@ -455,15 +458,10 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
     
         val firstTaskAttempts = taskScheduler.resourceOffers(offers).flatten
         firstTaskAttempts.foreach { task => logInfo(s"scheduled $task on ${task.executorId}") }
    -    assert(firstTaskAttempts.isEmpty)
    +    assert(firstTaskAttempts.size === 1)
    --- End diff --
    
    Can you also check that the executor ID is executor4?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r92079984
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala ---
    @@ -17,10 +17,322 @@
     
     package org.apache.spark.scheduler
     
    -import org.apache.spark.{SparkConf, SparkFunSuite}
    +import org.mockito.Mockito.when
    +import org.scalatest.BeforeAndAfterEach
    +import org.scalatest.mock.MockitoSugar
    +
    +import org.apache.spark._
     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 beforeEach(): Unit = {
    +    conf = new SparkConf().setAppName("test").setMaster("local")
    +      .set(config.BLACKLIST_ENABLED.key, "true")
    +    scheduler = mockTaskSchedWithConf(conf)
    +
    +    clock.setTime(0)
    +    blacklist = new BlacklistTracker(conf, clock)
    +  }
    +
    +  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, 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 { id =>
    +      val actual = f(id)
    +      val exp = expected.contains(id)
    +      assert(actual === exp, raw"""for string "$id" """)
    +    }
    +  }
    +
    +  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)
    +  }
    +
    +  test("executors can be blacklisted with only a few failures per stage") {
    +    // 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 should not unblacklist the
    +    // executor.
    +    val failuresUntilBlacklisted = conf.get(config.MAX_FAILURES_PER_EXEC)
    +    var failuresSoFar = 0
    +    (0 until failuresUntilBlacklisted * 10).foreach { stageId =>
    +      val taskSetBlacklist = createTaskSetBlacklist(stageId)
    +      if (stageId % 2 == 0) {
    +        // fail every other task
    +        taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 0)
    +        failuresSoFar += 1
    +      }
    +      blacklist.updateBlacklistForSuccessfulTaskSet(stageId, 0, taskSetBlacklist.execToFailures)
    +      assert(failuresSoFar == stageId / 2 + 1)
    +      if (failuresSoFar < failuresUntilBlacklisted) {
    +        assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set())
    +      } else {
    +        assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1"))
    +      }
    +    }
    +  }
    +
    +  // If an executor has many task failures, but the task set ends up failing, it shouldn't be
    +  // counted against the executor.
    +  test("executors aren't blacklisted if task sets fail") {
    --- End diff --
    
    "executors aren't blacklisted as a result of tasks in failed task sets"?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/69482/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r92271406
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala ---
    @@ -157,8 +160,16 @@ abstract class SchedulerIntegrationSuite[T <: MockBackend: ClassTag] extends Spa
           }
           // When a job fails, we terminate before waiting for all the task end events to come in,
           // so there might still be a running task set.  So we only check these conditions
    -      // when the job succeeds
    -      assert(taskScheduler.runningTaskSets.isEmpty)
    +      // when the job succeeds.
    +      // When the final task of a taskset completes, we post
    +      // the event to the DAGScheduler event loop before we finish processing in the taskscheduler
    +      // thread.  Its possible the DAGScheduler thread processes the event, finishes the job,
    +      // and notifies the job waiter before our original thread in the task scheduler finishes
    +      // handling the event and marks the taskset as complete.  So its ok if we need to wait a
    +      // *little* bit longer for the original taskscheduler thread to finish up to deal w/ the race.
    +      eventually(timeout(1 second), interval(100 millis)) {
    +        assert(taskScheduler.runningTaskSets.isEmpty)
    --- End diff --
    
    https://github.com/apache/spark/pull/16270


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r71611788
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -0,0 +1,221 @@
    +/*
    + * 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
    +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_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] = 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(
    +      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.
    +    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())
    --- End diff --
    
    sorry somehow I missed your responses here.  Yeah so I decided to just use the same timeout, after thinking about it some more it seems totally reasonable to me.
    
    but I hadn't thought about preemption, that is a really good point.  this was partially addressed in SPARK-8167, for making sure that preemption didn't lead to job failure.  I can change it to ignore failures any time `TaskEndReason.countTowardsTaskFailure` is false (in addition to ignoring fetch failures).  I wonder if I should do special handling for more of the task end reasons?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r70728183
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala ---
    @@ -0,0 +1,282 @@
    +/*
    + * 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.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.util.ManualClock
    +
    +class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with MockitoSugar
    +    with LocalSparkContext {
    +
    +  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)
    +
    +  var blacklistTracker: BlacklistTracker = _
    +
    +  override def afterEach(): Unit = {
    +    if (blacklistTracker != null) {
    +      blacklistTracker.stop()
    +      blacklistTracker = null
    +    }
    +    super.afterEach()
    +  }
    +
    +  val allOptions = (('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 = {
    +    allOptions.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
    +  }
    +
    +  test("Blacklisting individual tasks") {
    +    val conf = new SparkConf().setAppName("test").setMaster("local")
    +      .set("spark.ui.enabled", "false")
    +      .set("spark.scheduler.executorTaskBlacklistTime", "1000")
    +    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)
    +    val tsm = new TaskSetManager(scheduler, Some(blacklistTracker), taskSet, 4, clock)
    +    tsm.updateBlacklistForFailedTask("hostA", "1", 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(blacklistTracker.nodeBlacklist() === Set())
    +    assertEquivalentToSet(blacklistTracker.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,
    +    // 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)
    +    // 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 badPart = (index == 0 || index == 1)
    +        val taskExp = (badExec && badPart)
    +        assert(
    +          tsm.isExecutorBlacklistedForTask(executor, index) === taskExp)
    +        val executorExp = badExec
    +        assert(tsm.isExecutorBlacklistedForTaskSet(executor) === executorExp)
    +      }
    +    }
    +    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())
    +
    +    // when the stage completes successfully, now there is sufficient evidence we've got
    +    // bad executors and node
    +    blacklistTracker.taskSetSucceeded(tsm.execToFailures, scheduler)
    +    assert(blacklistTracker.nodeBlacklist() === Set("hostA"))
    +    assertEquivalentToSet(blacklistTracker.isNodeBlacklisted(_), Set("hostA"))
    +    assertEquivalentToSet(blacklistTracker.isExecutorBlacklisted(_), Set("1", "2"))
    +
    +    clock.advance(blacklistTracker.EXECUTOR_RECOVERY_MILLIS + 1)
    +    blacklistTracker.expireExecutorsInBlacklist()
    +    assert(blacklistTracker.nodeBlacklist() === Set())
    +    assertEquivalentToSet(blacklistTracker.isNodeBlacklisted(_), Set())
    +    assertEquivalentToSet(blacklistTracker.isExecutorBlacklisted(_), Set())
    +  }
    +
    +  def trackerFixture: (BlacklistTracker, TaskSchedulerImpl) = {
    +     val conf = new SparkConf().setAppName("test").setMaster("local")
    +      .set("spark.ui.enabled", "false")
    +      .set("spark.scheduler.executorTaskBlacklistTime", "1000")
    +    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)
    +    (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 =>
    +      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"))
    +  }
    +
    +  // 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 =>
    +      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())
    +  }
    +
    +  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)
    +      val taskSet = FakeTask.createTaskSet(4, stageId, 0)
    +      val tsm = new TaskSetManager(scheduler, Some(tracker), taskSet, 4, clock)
    +      (0 until 4).foreach { partition =>
    +        tsm.updateBlacklistForFailedTask("hostA", "1", partition)
    +      }
    +      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(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
    +        // stages
    +        tracker.taskSetFailed(stageId)
    +        assertEquivalentToSet(tracker.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, Some(tracker), taskSet0, 4, clock)
    +    (0 until 4).foreach { partition =>
    +      tsm0.updateBlacklistForFailedTask("hostA", "1", partition)
    +    }
    +    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 =>
    +      tsm1.updateBlacklistForFailedTask("hostA", "2", partition)
    +    }
    +    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)
    +    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
    +    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("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))
    --- End diff --
    
    `===` (also double check other asserts, just in case)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r86607623
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala ---
    @@ -17,10 +17,299 @@
     
     package org.apache.spark.scheduler
     
    -import org.apache.spark.{SparkConf, SparkFunSuite}
    +import org.mockito.Mockito.when
    +import org.scalatest.BeforeAndAfterEach
    +import org.scalatest.mock.MockitoSugar
    +
    +import org.apache.spark._
     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
    +  }
     
    -class BlacklistTrackerSuite extends SparkFunSuite {
    +  def createTaskSetBlacklist(stageId: Int = 0): TaskSetBlacklist = {
    +    new TaskSetBlacklist(conf, stageId, clock)
    +  }
    +
    +  def configureBlacklistAndScheduler(confs: (String, String)*): Unit = {
    --- End diff --
    
    should this just be in a beforeEach()? I know it's not needed for the old tests but seems like it doesn't do any harm?
    
    Also eliminate confs, since it doesn't seem to ever be used?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r72524536
  
    --- 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")
    +        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 (executorIdToBlacklistStatus.nonEmpty) {
    +        nextExpiryTime = executorIdToBlacklistStatus.map{_._2.expiryTime}.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(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
    +        executorIdToBlacklistStatus.put(exec, BlacklistedExecutor(node, expiryTime))
    +        val blacklistedExecsOnNode = nodeToFailedExecs.getOrElseUpdate(node, HashSet[String]())
    +        blacklistedExecsOnNode += exec
    +        executorIdToFailureCount.remove(exec)
    +        if (expiryTime < nextExpiryTime) {
    +          nextExpiryTime = expiryTime
    +        }
    +
    +        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)
    +      }
    +    }
    +  }
    +
    +  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 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.
    +    executorIdToFailureCount -= executorId
    --- End diff --
    
    I didn't understand your comment here -- why not remove it from executorIdToBlacklistStatus?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r72571864
  
    --- 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")
    +        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 (executorIdToBlacklistStatus.nonEmpty) {
    +        nextExpiryTime = executorIdToBlacklistStatus.map{_._2.expiryTime}.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(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
    +        executorIdToBlacklistStatus.put(exec, BlacklistedExecutor(node, expiryTime))
    +        val blacklistedExecsOnNode = nodeToFailedExecs.getOrElseUpdate(node, HashSet[String]())
    +        blacklistedExecsOnNode += exec
    +        executorIdToFailureCount.remove(exec)
    +        if (expiryTime < nextExpiryTime) {
    +          nextExpiryTime = expiryTime
    +        }
    +
    +        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)
    +      }
    +    }
    +  }
    +
    +  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 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.
    +    executorIdToFailureCount -= executorId
    --- End diff --
    
    Ah I wrote a vague comment about a vague comment :).  I meant: it looks like executorIdToBlacklistStatus isn't used when deciding which hosts to blacklist (looks like only nodeToFailedExecs is?) so seems like the executor could be removed from that map without messing up the host blacklist.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r72540910
  
    --- 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
    --- End diff --
    
    docstring here? "Time when the next blacklist will expire. Used to avoid checking for expired blacklist entries when none will have expired."


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #62352 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62352/consoleFull)** for PR 14079 at commit [`351a9a7`](https://github.com/apache/spark/commit/351a9a7e2893a0b90c57233d5e44a52c147bb2a8).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r74333404
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala ---
    @@ -770,9 +794,19 @@ 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())
    +
    +    // 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
    --- End diff --
    
    Cool thanks for fixing this!!


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r77888237
  
    --- Diff: docs/configuration.md ---
    @@ -1209,6 +1209,80 @@ Apart from these, the following properties are also available, and may be useful
       </td>
     </tr>
     <tr>
    +  <td><code>spark.blacklist.enabled</code></td>
    +  <td>
    +    <code>true</code> in cluster mode; <br/>
    +    <code>false</code> in local mode
    +  </td>
    +  <td>
    +    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.
    +  </td>
    +</tr>
    +<tr>
    +  <td><code>spark.blacklist.timeout</code></td>
    +  <td>1h</td>
    +  <td>
    +    (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.
    +  </td>
    +</tr>
    +<tr>
    +  <td><code>spark.blacklist.task.maxTaskAttemptsPerExecutor</code></td>
    +  <td>1</td>
    +  <td>
    +    (Experimental) For a given task, how many times it can be retried on one executor before the
    +    executor is blacklisted for that task.
    +  </td>
    +</tr>
    +<tr>
    +  <td><code>spark.blacklist.task.maxTaskAttemptsPerNode</code></td>
    +  <td>2</td>
    +  <td>
    +    (Experimental) For a given task, how many times it can be retried on one node, before the entire
    +    node is blacklisted for that task.
    +  </td>
    +</tr>
    +<tr>
    +  <td><code>spark.blacklist.stage.maxFailedTasksPerExecutor</code>
    +  <td>2</td>
    +  <td>
    +    (Experimental) How many different tasks must fail on one executor, within one stage, before the
    +    executor is blacklisted for that stage.
    +  </td>
    +</tr>
    +<tr>
    +  <td><code>spark.blacklist.stage.maxFailedExecutorsPerNode</code></td>
    +  <td>2</td>
    +  <td>
    +    (Experimental) How many different executors are marked as failed for a given stage, before the
    +    entire node is marked as failed for the stage.
    --- End diff --
    
    should be marked as blacklisted for the stage


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #62630 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62630/consoleFull)** for PR 14079 at commit [`f0de0db`](https://github.com/apache/spark/commit/f0de0db8c54ac2dc66b0ab59b1b6c155a6775014).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r77275570
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -0,0 +1,395 @@
    +/*
    + * 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.{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[scheduler] 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.
    +   */
    +  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
    +   * 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
    +  }
    +}
    +
    +
    +private[scheduler] object BlacklistTracker extends Logging {
    +
    +  private val DEFAULT_TIMEOUT = "1h"
    +
    +  /**
    +   * 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:
    +   *   - off for local mode
    +   *   - on for distributed modes (including local-cluster)
    +   */
    +  def isBlacklistEnabled(conf: SparkConf): Boolean = {
    +    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 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
    +            } 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
    +            !Utils.isLocalMaster(conf)
    +        }
    +    }
    +  }
    +
    +  def getBlacklistTimeout(conf: SparkConf): Long = {
    +    conf.get(config.BLACKLIST_TIMEOUT_CONF).getOrElse {
    +      conf.get(config.BLACKLIST_LEGACY_TIMEOUT_CONF).getOrElse {
    +        Utils.timeStringAsMs(DEFAULT_TIMEOUT)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * 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 */
    +private[scheduler] final class ExecutorFailuresInTaskSet(val node: String) {
    --- End diff --
    
    Scala style (/ convention in Spark, I think) is to put  classes like this in their own files, partially (IMO) because it's easier to find things that way.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r72538489
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala ---
    @@ -677,8 +702,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(
    --- End diff --
    
    maybe "(executor %s)" (favoring parens / verbosity since this is a message that does commonly pop up to users in the console


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r74327484
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -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: Option[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))
    --- End diff --
    
    I was thinking of this part of the Scale style guide: http://docs.scala-lang.org/style/indentation.html#line_wrapping
    
    I know Spark doesn't follow this super consistently, but I do think it helps readability (esp. in cases like this where there are some long-ish individual parameters, so it's hard to scan the list), so good to do IMO where we remember.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r92075850
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -17,10 +17,275 @@
     
     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.Utils
    +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 = new  HashMap[String, ExecutorFailureList]()
    +  val executorIdToBlacklistStatus = new HashMap[String, BlacklistedExecutor]()
    +  val nodeIdToBlacklistExpiryTime = new HashMap[String, Long]()
    +  /**
    +   * An immutable copy of the set of nodes that are currently blacklisted.  Kept in an
    +   * AtomicReference to make [[nodeBlacklist()]] thread-safe.
    +   */
    +  private val _nodeBlacklist = new AtomicReference[Set[String]](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 clean up the list of blacklisted executors once an executor has
    +   * been blacklisted for BLACKLIST_TIMEOUT_MILLIS.
    +   */
    +  val nodeToBlacklistedExecs = new HashMap[String, HashSet[String]]()
    +
    +  /**
    +   * Un-blacklists executors and nodes that have been blacklisted for at least
    +   * BLACKLIST_TIMEOUT_MILLIS
    +   */
    +  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"for those executors has timed out")
    +        execsToUnblacklist.foreach { exec =>
    +          val status = executorIdToBlacklistStatus.remove(exec).get
    +          val failedExecsOnNode = nodeToBlacklistedExecs(status.node)
    +          failedExecsOnNode.remove(exec)
    +          if (failedExecsOnNode.isEmpty) {
    +            nodeToBlacklistedExecs.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")
    +        nodeIdToBlacklistExpiryTime --= nodesToUnblacklist
    +        _nodeBlacklist.set(nodeIdToBlacklistExpiryTime.keySet.toSet)
    +      }
    +      updateNextExpiryTime()
    +    }
    +  }
    +
    +  private def updateNextExpiryTime(): Unit = {
    +    val execMinExpiry = if (executorIdToBlacklistStatus.nonEmpty) {
    +      executorIdToBlacklistStatus.map{_._2.expiryTime}.min
    +    } else {
    +      Long.MaxValue
    +    }
    +    val nodeMinExpiry = if (nodeIdToBlacklistExpiryTime.nonEmpty) {
    +      nodeIdToBlacklistExpiryTime.values.min
    +    } else {
    +      Long.MaxValue
    +    }
    +    nextExpiryTime = math.min(execMinExpiry, nodeMinExpiry)
    +  }
    +
    +
    +  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
    --- End diff --
    
    To mitigate part of my confusion in the comment above (about which expiry times are used where), what about moving this to line 154 (right before "// If this..."), which is closer to where it's used, and renaming to the more verbose expiryTimeForNewBlacklists or similar?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r92231466
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala ---
    @@ -157,8 +160,16 @@ abstract class SchedulerIntegrationSuite[T <: MockBackend: ClassTag] extends Spa
           }
           // When a job fails, we terminate before waiting for all the task end events to come in,
           // so there might still be a running task set.  So we only check these conditions
    -      // when the job succeeds
    -      assert(taskScheduler.runningTaskSets.isEmpty)
    +      // when the job succeeds.
    +      // When the final task of a taskset completes, we post
    +      // the event to the DAGScheduler event loop before we finish processing in the taskscheduler
    +      // thread.  Its possible the DAGScheduler thread processes the event, finishes the job,
    +      // and notifies the job waiter before our original thread in the task scheduler finishes
    +      // handling the event and marks the taskset as complete.  So its ok if we need to wait a
    +      // *little* bit longer for the original taskscheduler thread to finish up to deal w/ the race.
    +      eventually(timeout(1 second), interval(100 millis)) {
    +        assert(taskScheduler.runningTaskSets.isEmpty)
    --- End diff --
    
    Cool sounds good re: separate fix.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r71524840
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -0,0 +1,221 @@
    +/*
    + * 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
    +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_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] = 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(
    +      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.
    +    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())
    --- End diff --
    
    actually I guess that could be bad in the case of like YARN preemption, I haven't seen the scheduler take that into account does it?  I know the yarn allocator does for executor failures but I assume scheduler just see executor lost and fails the tasks as normal?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #62134 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62134/consoleFull)** for PR 14079 at commit [`c22aaad`](https://github.com/apache/spark/commit/c22aaad76f07cbe58ea455d18959470e7afb1498).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/65939/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r71411885
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -0,0 +1,221 @@
    +/*
    + * 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
    +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_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] = 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(
    +      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.
    +    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())
    --- End diff --
    
    oh good point.  this design was somewhat intentional -- to deal w/ intermittent flakiness, you wouldn't blacklist a node if eg. you had one executor get blacklisted on it 3 times, but with long periods in between.  I see though that this is probably undesirable with DA.
    
    Its actually a really good point, that with DA node blacklisting is important even with just one executor per node.
    
    We could either:
    1) keep it as it is now, so each new executor that DA adds has to get re-blacklisted. 
    2) always blacklist the node after N blacklisted executors, regardless of the time between
    3) add a timeout for how much time can go between blacklisted executors for it to count.  Then we've got to either add another configuration or re-use one of the existing ones.
    
    I don't like the added configuration burden of 3 ... I'm leaning towards 2.  Though maybe I'm forced to use 3 just because otherwise we track failed executors for all time, which I guess would be a memory leak, though hopefully an extremely slow one.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r71401929
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -0,0 +1,221 @@
    +/*
    + * 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
    +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_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] = 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(
    +      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.
    +    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)
    --- End diff --
    
    this is going to throw a "java.util.NoSuchElementException: key not found" exception is the executor is dead because scheduler removes it


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r71431590
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -0,0 +1,221 @@
    +/*
    + * 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
    +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_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] = 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(
    +      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.
    +    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())
    --- End diff --
    
    well its more then just DA, its actually anytime an executor is lost, like I mention above with scheduler.getHostForExecutor, this was static number of executors, one of them was killed, the blacklisting doesn't get propagated after the stage completes.
    
    we do track dead executors, just not in the scheduler, they are in StorageStatusListener.
    
    As far as the time between, I thought that is what the BLACKLIST_EXPIRY_TIMEOUT_CONF config was for.  yes there could be a long time between first and second failure, then it gets blacklisted, but if long time before third it would be pulled off.  The same issue exists with tasks on executors, right?  It just depends on how many tasks I have.  It could be that a task doesn't run on an executor for a long time, but then in next stage it does.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by squito <gi...@git.apache.org>.
Github user squito commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    also just realized that I forgot about @kayousterhout 's comment to add in checks on the invariants for the confs -- I've added that now as well.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by squito <gi...@git.apache.org>.
Github user squito commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    thanks @kayousterhout !  appreciate all the time you've spent helping out on this issue.
    
    merged to master


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r79911314
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -0,0 +1,393 @@
    +/*
    + * 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.{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[scheduler] 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.
    +   */
    +  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
    +   * 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
    +  }
    +}
    +
    +
    +private[scheduler] object BlacklistTracker extends Logging {
    +
    +  private val DEFAULT_TIMEOUT = "1h"
    +
    +  /**
    +   * 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:
    +   *   - off for local mode
    +   *   - on for distributed modes (including local-cluster)
    +   */
    +  def isBlacklistEnabled(conf: SparkConf): Boolean = {
    +    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 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
    +            } 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
    +            !Utils.isLocalMaster(conf)
    +        }
    +    }
    +  }
    +
    +  def getBlacklistTimeout(conf: SparkConf): Long = {
    +    conf.get(config.BLACKLIST_TIMEOUT_CONF).getOrElse {
    +      conf.get(config.BLACKLIST_LEGACY_TIMEOUT_CONF).getOrElse {
    +        Utils.timeStringAsMs(DEFAULT_TIMEOUT)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * 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 (maxNodeAttempts >= maxTaskFailures) {
    +        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}")
    +      }
    +    }
    +
    +  }
    +}
    +
    +/** 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"
    +  }
    +}
    +
    +/**
    + * 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)
    --- End diff --
    
    good point, moved this and `ExecutorFailureList` as well.  I'd prefer to keep it as a case class since there are uses like `executorIdToBlacklistStatus.filter(_._2.expiryTime < now).keys` and I think that is much clearer with the `expiryTime` in there.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r77295487
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.BeforeAndAfterEach
    +import org.scalatest.mock.MockitoSugar
    +
    +import org.apache.spark._
    +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 blacklistTracker: BlacklistTracker = _
    +
    +  override def afterEach(): Unit = {
    +    if (blacklistTracker != null) {
    +      blacklistTracker = null
    +    }
    +    super.afterEach()
    +  }
    +
    +  val allOptions = (('A' to 'Z').map("host" + _) ++ (1 to 100).map{_.toString}).toSet
    --- End diff --
    
    allExecutorAndHostIds? allExecutorAndHostNames?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r77297537
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.BeforeAndAfterEach
    +import org.scalatest.mock.MockitoSugar
    +
    +import org.apache.spark._
    +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 blacklistTracker: BlacklistTracker = _
    +
    +  override def afterEach(): Unit = {
    +    if (blacklistTracker != null) {
    +      blacklistTracker = null
    +    }
    +    super.afterEach()
    +  }
    +
    +  val allOptions = (('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 = {
    +    allOptions.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
    +  }
    +
    +  test("Blacklisting individual tasks") {
    --- End diff --
    
    I wonder if this should be a test for the TaskSetManager (so in TaskSetManagerSuite), where you pass in a mock'ed blacklist tracker, and make sure that no calls get made on that blacklist tracker?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r70004202
  
    --- 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
    +   * the executor is blacklisted overall (or anything to do with the node the executor is on).
    +   */
    +  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)
    +  }
    +
    +  def nodeBlacklist(): Set[String] = {
    +    _nodeBlacklist.get()
    +  }
    +
    +  def isNodeBlacklisted(node: String): Boolean = {
    +    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 previous
    +    // 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.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
    +      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.
    +   */
    +  def isExecutorBlacklisted(
    +      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 {
    --- End diff --
    
    Wonder if something like this isn't easier to follow:
    
    ```
        stageIdToExecToFailures.get(stageId)
          .flatMap(_.get(executorId))
          .map(_.failuresByTask.contains(indexInTaskSet))
          .getOrElse(false)
    ```



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #69417 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/69417/consoleFull)** for PR 14079 at commit [`fd57d86`](https://github.com/apache/spark/commit/fd57d868561c552ea171b4d098ebc2473abb1de6).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r72527507
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -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: Option[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 = isLocal)
    --- End diff --
    
    indentation


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/65919/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r86599228
  
    --- Diff: 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, ThreadUtils, Utils}
    +import org.apache.spark.util.{AccumulatorV2, SystemClock, ThreadUtils, Utils}
    --- End diff --
    
    Looks like this isn't used


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by squito <gi...@git.apache.org>.
Github user squito commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    oops, thanks for letting me know @zsxwing , I just submitted https://github.com/apache/spark/pull/16298


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #70194 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/70194/testReport)** for PR 14079 at commit [`f249b00`](https://github.com/apache/spark/commit/f249b00e3eb64bf35ab836fa4b89eb961a9511a5).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class EventTimeStats(var max: Long, var min: Long, var sum: Long, var count: Long) `
      * `class EventTimeStatsAccum(protected var currentStats: EventTimeStats = EventTimeStats.zero)`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r77297069
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.BeforeAndAfterEach
    +import org.scalatest.mock.MockitoSugar
    +
    +import org.apache.spark._
    +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 blacklistTracker: BlacklistTracker = _
    +
    +  override def afterEach(): Unit = {
    +    if (blacklistTracker != null) {
    +      blacklistTracker = null
    +    }
    +    super.afterEach()
    +  }
    +
    +  val allOptions = (('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 = {
    +    allOptions.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
    +  }
    +
    +  test("Blacklisting individual tasks") {
    +    val conf = new SparkConf().setAppName("test").setMaster("local")
    +      .set(config.BLACKLIST_ENABLED.key, "true")
    +    val scheduler = mockTaskSchedWithConf(conf)
    +    // 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)
    +    tsm.updateBlacklistForFailedTask("hostA", "1", 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(blacklistTracker.nodeBlacklist() === Set())
    +    assertEquivalentToSet(blacklistTracker.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,
    +    // 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)
    +    // 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 badPart = (index == 0 || index == 1)
    +        val taskExp = (badExec && badPart)
    +        assert(
    +          tsm.isExecutorBlacklistedForTask(executor, index) === taskExp)
    +        val executorExp = badExec
    +        assert(tsm.isExecutorBlacklistedForTaskSet(executor) === executorExp)
    +      }
    +    }
    +    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())
    +
    +    // 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()
    +  }
    +
    +  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)
    +    blacklistTracker = new BlacklistTracker(conf, clock)
    +    (blacklistTracker, scheduler)
    --- End diff --
    
    would it make sense to have the scheduler also be a class variable, and then this function could be something like configureTrackerAndScheduler(), and each test could use the class variables directly (rather than creating local variables that they use)? The local variables seem prone to errors in future PRs, where someone doesn't realize that the blacklist tracker is / isn't getting nulled at the end properly by afterEach()


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r84148084
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala ---
    @@ -282,6 +331,211 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
         assert(!failedTaskSet)
       }
     
    +  test("scheduled tasks obey task and stage blacklists") {
    --- End diff --
    
    this test case really should have been part of the last PR, oops.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by squito <gi...@git.apache.org>.
Github user squito commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Jenkins, retest this please


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r92103358
  
    --- Diff: docs/configuration.md ---
    @@ -1339,6 +1347,28 @@ Apart from these, the following properties are also available, and may be useful
       </td>
     </tr>
     <tr>
    +  <td><code>spark.blacklist.application.maxFailedTasksPerExecutor</code></td>
    +  <td>2</td>
    +  <td>
    +    (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
    +    <code>spark.blacklist.timeout</code>.  Note that with dynamic allocation, though, the executors
    +    may get marked as idle and be reclaimed by the cluster manager.
    +  </td>
    +</tr>
    +<tr>
    +  <td><code>spark.blacklist.application.maxFailedExecutorsPerNode</code></td>
    +  <td>2</td>
    +  <td>
    +    (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
    +    <code>spark.blacklist.timeout</code>.  Note that with dynamic allocation, though, the executors
    +    may get marked as idle and be reclaimed by the cluster manager.
    --- End diff --
    
    the executors on the node may


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r86599669
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -51,13 +50,27 @@ 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 val blacklistTracker: Option[BlacklistTracker],
         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),
    --- End diff --
    
    Ooohh I see this is fine then


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r77278627
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala ---
    @@ -592,34 +610,59 @@ 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 = {
    -
    -    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))
    -      }
    -    }
    +  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) {
    +        // 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 {
    +            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 (executors.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.")
    +        // 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) {
    --- End diff --
    
    why not check this at the beginning, with the isZombie check?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/62621/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r86602962
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala ---
    @@ -39,9 +43,12 @@ 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)
    --- End diff --
    
    I think it would be better to remove this, as per comment in ExecutorFailuresInTaskSet.scala


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/67271/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r72538331
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala ---
    @@ -611,15 +620,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 (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.")
    +      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
    +          } else {
    +            execs.forall { exec =>
    --- End diff --
    
    and "Check if the task can run on any of the executors"


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r77273162
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -0,0 +1,393 @@
    +/*
    + * 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.{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[scheduler] 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.
    +   */
    +  private[scheduler] var nextExpiryTime: Long = Long.MaxValue
    --- End diff --
    
    I think private[scheduler] is redundant here since the class is private[scheduler]


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #64723 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/64723/consoleFull)** for PR 14079 at commit [`c78964f`](https://github.com/apache/spark/commit/c78964fb4dad0662f1670634126d538aa71b1cf5).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #69092 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/69092/consoleFull)** for PR 14079 at commit [`a8d922f`](https://github.com/apache/spark/commit/a8d922fb267fa3f794b433ff33ce0b0368c36bf5).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r86649751
  
    --- Diff: yarn/src/test/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackendSuite.scala ---
    @@ -0,0 +1,57 @@
    +/*
    + * 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.cluster
    +
    +import org.mockito.Mockito.when
    +import org.scalatest.mock.MockitoSugar
    +
    +import org.apache.spark.{LocalSparkContext, SparkContext, SparkFunSuite}
    +import org.apache.spark.scheduler.TaskSchedulerImpl
    +import org.apache.spark.serializer.JavaSerializer
    +
    +class YarnSchedulerBackendSuite extends SparkFunSuite with MockitoSugar with LocalSparkContext {
    +
    +  test("RequestExecutors reflects node blacklist and is serializable") {
    +    sc = new SparkContext("local", "YarnSchedulerBackendSuite")
    +    val sched = mock[TaskSchedulerImpl]
    +    when(sched.sc).thenReturn(sc)
    +    val yarnSchedulerBackend = new YarnSchedulerBackend(sched, sc) {
    +      def setHostToLocalTaskCount(hostToLocalTaskCount: Map[String, Int]): Unit = {
    +        this.hostToLocalTaskCount = hostToLocalTaskCount
    +      }
    +    }
    +    val ser = new JavaSerializer(sc.conf).newInstance()
    +    for {
    +      blacklist <- IndexedSeq(Set[String](), Set("a", "b", "c"))
    +      numRequested <- 0 until 10
    +      hostToLocalCount <- IndexedSeq(
    +        Map[String, Int](),
    +        Map("a" -> 1, "b" -> 2)
    +      )
    +    } {
    +      yarnSchedulerBackend.setHostToLocalTaskCount(hostToLocalCount)
    +      when(sched.nodeBlacklist()).thenReturn(blacklist)
    +      val req = yarnSchedulerBackend.prepareRequestExecutors(numRequested)
    --- End diff --
    
    can this mock (and verify) yarnSchedulerEndpointRef to avoid the extra prepareRequestExecutors helper method?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r72485685
  
    --- Diff: 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.blacklist.enabled")
    +    .booleanConf
    +    .createOptional
    +
    +  private[spark] val MAX_TASK_ATTEMPTS_PER_NODE =
    +    ConfigBuilder("spark.blacklist.maxTaskAttemptsPerNode")
    +    .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")
    --- End diff --
    
    As discussed in the design doc, I think this would be slightly more clear (and the hierarchy of configurations would be more clear) if this were spark.blacklist.stage.maxFailedTasksPerExecutor


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r72540337
  
    --- Diff: yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala ---
    @@ -217,18 +219,34 @@ 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
    --- End diff --
    
    a set of blacklisted nodes, which is passed in to avoid allocating new containers on them.  It will be used to update the application manager (? right terminology?) blacklist.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #62894 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62894/consoleFull)** for PR 14079 at commit [`c9e3662`](https://github.com/apache/spark/commit/c9e3662dd468fe262fcb663ca1898cac0029f8d7).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #69417 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/69417/consoleFull)** for PR 14079 at commit [`fd57d86`](https://github.com/apache/spark/commit/fd57d868561c552ea171b4d098ebc2473abb1de6).
     * This patch **fails from timeout after a configured wait of \`250m\`**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r92267462
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -678,4 +716,13 @@ private[spark] object TaskSchedulerImpl {
     
         retval.toList
       }
    +
    +  private def maybeCreateBlacklistTracker(conf: SparkConf): Option[BlacklistTracker] = {
    --- End diff --
    
    the scope becomes a little wider, though -- its only used in `TaskSchedulerImpl`, and I'd have to make it `private[scheduler]`.  If it were there, I worry that it might look like a way to get a hold of the global singleton, like the `getOrCreate` methods on other objects.
    
    I see it both ways, but think I feel a bit better about leaving it here so its `private`.  what do you think?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r79916381
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala ---
    @@ -592,34 +610,59 @@ 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 = {
    -
    -    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))
    -      }
    -    }
    +  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,
    --- End diff --
    
    good point, it is not


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #65783 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/65783/consoleFull)** for PR 14079 at commit [`18ef5c6`](https://github.com/apache/spark/commit/18ef5c6874f5f8fb20ffce238dd2d6e7d7849ca9).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #67271 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/67271/consoleFull)** for PR 14079 at commit [`162cb0d`](https://github.com/apache/spark/commit/162cb0d4253cddddacf93d7e6985c01d730ec27e).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #61931 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/61931/consoleFull)** for PR 14079 at commit [`cf58374`](https://github.com/apache/spark/commit/cf5837410818dae093ef15617cb42336a14408db).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r70005272
  
    --- 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
    +   * the executor is blacklisted overall (or anything to do with the node the executor is on).
    +   */
    +  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)
    +  }
    +
    +  def nodeBlacklist(): Set[String] = {
    +    _nodeBlacklist.get()
    +  }
    +
    +  def isNodeBlacklisted(node: String): Boolean = {
    +    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 previous
    +    // 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.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
    +      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.
    +   */
    +  def isExecutorBlacklisted(
    +      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
    +    }
    +  }
    +
    +  def isNodeBlacklisted(
    +      node: String,
    +      stageId: Int,
    +      indexInTaskSet: Int): Boolean = {
    +    stageIdToNodeBlacklistedTasks.get(stageId).flatMap { nodeToFailures =>
    +      nodeToFailures.get(node).map{_.contains(indexInTaskSet)}
    +    }.getOrElse(false)
    +  }
    +
    +  def removeExecutor(executorId: String): Unit = {
    +    executorIdToBlacklistExpiryTime -= executorId
    +    executorIdToFailureCount -= executorId
    +    stageIdToExecToFailures.values.foreach { execFailureOneStage =>
    +      execFailureOneStage -= executorId
    +    }
    +  }
    +}
    +
    +
    +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:
    +   * 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
    +    }
    +  }
    +
    +  def getBlacklistExpiryTime(conf: SparkConf): Long = {
    +    conf.getTimeAsMs(BlacklistTracker.EXPIRY_TIMEOUT_CONF,
    +      conf.get(BlacklistTracker.LEGACY_TIMEOUT_CONF, (60 * 60 * 1000).toString))
    --- End diff --
    
    `"1h"` instead of doing math.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/69428/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #63989 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/63989/consoleFull)** for PR 14079 at commit [`f0428b4`](https://github.com/apache/spark/commit/f0428b46e7de9837fa30850be85b7fb482191a48).
     * This patch **fails to build**.
     * This patch **does not merge cleanly**.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r77893351
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala ---
    @@ -51,37 +54,68 @@ 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 > min(spark.blacklist.maxTaskFailuresPerNode, nExecutorsPerHost)
    --- End diff --
    
    comment out of date spark.blacklist.maxTaskFailuresPerNode doesn't exist.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/64723/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by squito <gi...@git.apache.org>.
Github user squito commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    ping @kayousterhout.  I've merged in the other changes so this is up to date now.
    I also did another pass and did a bit of minor cleanup and commenting, I think in line with stuff we've discussed elsewhere.  I hope this is in pretty good shape now.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/70194/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r88121951
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala ---
    @@ -84,7 +85,7 @@ private[spark] class TaskSetManager(
       var totalResultSize = 0L
       var calculatedTasks = 0
     
    -  private val taskSetBlacklistHelperOpt: Option[TaskSetBlacklist] = {
    +  private[scheduler] val taskSetBlacklistHelperOpt: Option[TaskSetBlacklist] = {
    --- End diff --
    
    yeah good question.  I wondered this myself ... I thought about instead making an `Option[(TaskSetBlacklist, BlacklistTracker)]` but it just made all the other calls ugly.  I'll make it check whether blacklistTracker is set instead.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #64499 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/64499/consoleFull)** for PR 14079 at commit [`5fdfe49`](https://github.com/apache/spark/commit/5fdfe49d16996e6317b291e469656f68d1def3a8).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r72538818
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala ---
    @@ -611,15 +620,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 (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.")
    +      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
    +          } else {
    +            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) cannot " +
    +            s"run anywhere due to node and executor blacklist.")
    --- End diff --
    
    Maybe refer to the config options here? Anticipating confused users...


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #64215 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/64215/consoleFull)** for PR 14079 at commit [`fc45f5b`](https://github.com/apache/spark/commit/fc45f5b2e2fc38aff0714f1465f03f5e0ba16e01).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by squito <gi...@git.apache.org>.
Github user squito commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    for anyone else who might be watching, I've carved out part of this change here: https://github.com/apache/spark/pull/15249


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r75504152
  
    --- Diff: docs/configuration.md ---
    @@ -1178,6 +1178,80 @@ Apart from these, the following properties are also available, and may be useful
       </td>
     </tr>
     <tr>
    +  <td><code>spark.blacklist.enabled</code></td>
    +  <td>
    +    <code>true</code> in cluster mode; <br/>
    +    <code>false</code> in local mode
    +  </td>
    +  <td>
    +    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.
    +  </td>
    +</tr>
    +<tr>
    +  <td><code>spark.blacklist.timeout</code></td>
    +  <td>1h</td>
    +  <td>
    +    (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.
    +  </td>
    +</tr>
    +<tr>
    +  <td><code>spark.blacklist.task.maxTaskAttemptsPerExecutor</code></td>
    +  <td>2</td>
    --- End diff --
    
    default is actually 1


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r72527248
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -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](
    --- End diff --
    
    I know it's annoying (or requires creating a helper for TaskSetManagers) for the tests, but can blacklistTracker just be private? Seems cleaner and unnecessary to expose it.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r72499789
  
    --- 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
    --- End diff --
    
    Can you clean up the comments here so they're docstrings (for this one and the next one) and start with a capital letter + (when a sentence) end with a period, for all of the ones in this file?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r70727301
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala ---
    @@ -611,16 +614,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.")
    +      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
    --- End diff --
    
    Another case of using `execs.exists`?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r72541175
  
    --- 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())
    --- End diff --
    
    add comment here saying that you keep a copy of the node blacklist in an atomicref so that it can be read in a sep. thread?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r76682839
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -0,0 +1,395 @@
    +/*
    + * 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.{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[scheduler] 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.
    +   */
    +  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
    +   * 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
    +  }
    +}
    +
    +
    +private[scheduler] object BlacklistTracker extends Logging {
    +
    +  private val DEFAULT_TIMEOUT = "1h"
    +
    +  /**
    +   * 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:
    +   *   - off for local mode
    +   *   - on for distributed modes (including local-cluster)
    +   */
    +  def isBlacklistEnabled(conf: SparkConf): Boolean = {
    +    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 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
    +            } 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
    +            !Utils.isLocalMaster(conf)
    +        }
    +    }
    +  }
    +
    +  def getBlacklistTimeout(conf: SparkConf): Long = {
    +    conf.get(config.BLACKLIST_TIMEOUT_CONF).getOrElse {
    +      conf.get(config.BLACKLIST_LEGACY_TIMEOUT_CONF).getOrElse {
    +        Utils.timeStringAsMs(DEFAULT_TIMEOUT)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * 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 " +
    --- End diff --
    
    one failed disk is pretty specific, perhaps bad nodes or something more generic


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by squito <gi...@git.apache.org>.
Github user squito commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    @kayousterhout @tgravescs Sorry for the long-delay on my end, I have finally updated this.  I think I have addressed all the naming / commenting issues -- I'm going to do a bit more testing on it and then come back for a fresh pass to see if there are more things.
    
    Some important changes in this revisions:
    
    * timeouts of individual task failures, in application blacklisting only.  This is to avoid blacklisting an executor in a long-lived app when you've got a few random failures that are very far apart in time.  But everything still has its own timeout -- here's the comment I've put in the code to explain that:
    ```scala
          // 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.
    ```
    * config name changes.  Also, docs mark everything as "Experimental" except for spark.blacklist.enabled.  The idea is to convey that blacklisting will continue to be exported, but the specific knobs may go through another revision.
    * bug fix -- in some more testing, I discovered that I was blacklisting nodes whenever the *total* number of blacklisted executors was over the threshold, rather than checking the number of blacklisted executors on *that node*.  Fixed, cleaned up code a bit, added a test.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r77279586
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala ---
    @@ -800,6 +842,86 @@ 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.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; 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
    +    }
    +
    +    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.getOrElse(index, (0, 0))._1
    --- End diff --
    
    i would find this slightly more intuitive as ...AndExpiryTime.get(index).map(_._1).getOrElse(0)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by squito <gi...@git.apache.org>.
Github user squito commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Jenkins, retest this please


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #68694 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/68694/consoleFull)** for PR 14079 at commit [`cc3faaf`](https://github.com/apache/spark/commit/cc3faaf720148e2c92bb6858231929473b5ab19f).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/68694/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #67623 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/67623/consoleFull)** for PR 14079 at commit [`255e9b6`](https://github.com/apache/spark/commit/255e9b613b937b3875b728622da32ac6f1653c92).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r92274093
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -678,4 +716,13 @@ private[spark] object TaskSchedulerImpl {
     
         retval.toList
       }
    +
    +  private def maybeCreateBlacklistTracker(conf: SparkConf): Option[BlacklistTracker] = {
    --- End diff --
    
    Ah ok good point that seems like sufficient reason to leave it here.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/69433/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #62352 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62352/consoleFull)** for PR 14079 at commit [`351a9a7`](https://github.com/apache/spark/commit/351a9a7e2893a0b90c57233d5e44a52c147bb2a8).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/67623/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #64499 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/64499/consoleFull)** for PR 14079 at commit [`5fdfe49`](https://github.com/apache/spark/commit/5fdfe49d16996e6317b291e469656f68d1def3a8).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r92081435
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala ---
    @@ -157,8 +160,16 @@ abstract class SchedulerIntegrationSuite[T <: MockBackend: ClassTag] extends Spa
           }
           // When a job fails, we terminate before waiting for all the task end events to come in,
           // so there might still be a running task set.  So we only check these conditions
    -      // when the job succeeds
    -      assert(taskScheduler.runningTaskSets.isEmpty)
    +      // when the job succeeds.
    +      // When the final task of a taskset completes, we post
    +      // the event to the DAGScheduler event loop before we finish processing in the taskscheduler
    +      // thread.  Its possible the DAGScheduler thread processes the event, finishes the job,
    +      // and notifies the job waiter before our original thread in the task scheduler finishes
    +      // handling the event and marks the taskset as complete.  So its ok if we need to wait a
    +      // *little* bit longer for the original taskscheduler thread to finish up to deal w/ the race.
    +      eventually(timeout(1 second), interval(100 millis)) {
    +        assert(taskScheduler.runningTaskSets.isEmpty)
    --- End diff --
    
    Is this related to this PR? 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #70102 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/70102/consoleFull)** for PR 14079 at commit [`c422dd4`](https://github.com/apache/spark/commit/c422dd48318fe94ee26655437d58537ee57dd85c).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #67205 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/67205/consoleFull)** for PR 14079 at commit [`21907a5`](https://github.com/apache/spark/commit/21907a58d6e089940cf3a3a0f27b32460d43de2b).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #67207 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/67207/consoleFull)** for PR 14079 at commit [`0c57d9d`](https://github.com/apache/spark/commit/0c57d9d576ad35133322f80fc8acc8ed35a90396).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #62630 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62630/consoleFull)** for PR 14079 at commit [`f0de0db`](https://github.com/apache/spark/commit/f0de0db8c54ac2dc66b0ab59b1b6c155a6775014).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r72527481
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -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: Option[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))
    --- End diff --
    
    indentation (if all params don't fit on a single line, they should each have their own line)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r72284207
  
    --- Diff: core/src/main/scala/org/apache/spark/TaskEndReason.scala ---
    @@ -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
    --- End diff --
    
    the default in `TaskFailedReason` is true, so it will be true for `ExceptionFailure`, `UnknownReason`, `ExecutorLost` if caused by app (eg., an OOM), `TaskResultLost`, and `Resubmitted`.  I admit I'm not certain if those last two should count or not, but changing that behavior should probably be separate from this change.  I'm not changing the end behavior here, just trying to refactor it to make it easier to follow and handle correctly with the blacklist (it was special-cased before: https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala#L787)
    
    Or do you mean we should change it to a `val` instead, since its a constant?  then I'd change `TaskCommitDenied` to use a `val` as well.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by squito <gi...@git.apache.org>.
Github user squito commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    thanks for the feedback @tgravescs .  Thanks for pointing out the issue after executors have died, I updated the behavior and added a test case (I'll update the design doc shortly too).  I think I addressed the other comments, including finally adding the docs.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r77275061
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -0,0 +1,393 @@
    +/*
    + * 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.{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[scheduler] 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.
    +   */
    +  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
    +   * 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
    +  }
    +}
    +
    +
    +private[scheduler] object BlacklistTracker extends Logging {
    +
    +  private val DEFAULT_TIMEOUT = "1h"
    +
    +  /**
    +   * 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:
    +   *   - off for local mode
    +   *   - on for distributed modes (including local-cluster)
    +   */
    +  def isBlacklistEnabled(conf: SparkConf): Boolean = {
    +    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 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
    +            } 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
    +            !Utils.isLocalMaster(conf)
    +        }
    +    }
    +  }
    +
    +  def getBlacklistTimeout(conf: SparkConf): Long = {
    +    conf.get(config.BLACKLIST_TIMEOUT_CONF).getOrElse {
    +      conf.get(config.BLACKLIST_LEGACY_TIMEOUT_CONF).getOrElse {
    +        Utils.timeStringAsMs(DEFAULT_TIMEOUT)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * 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.
    --- End diff --
    
    why do the tests need this? Just worried about users finding this / using it without understanding what's happening.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r76685355
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -0,0 +1,395 @@
    +/*
    + * 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.{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[scheduler] 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.
    +   */
    +  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
    +   * 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
    +  }
    +}
    +
    +
    +private[scheduler] object BlacklistTracker extends Logging {
    +
    +  private val DEFAULT_TIMEOUT = "1h"
    +
    +  /**
    +   * 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:
    +   *   - off for local mode
    +   *   - on for distributed modes (including local-cluster)
    +   */
    +  def isBlacklistEnabled(conf: SparkConf): Boolean = {
    +    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 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
    +            } 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
    +            !Utils.isLocalMaster(conf)
    +        }
    +    }
    +  }
    +
    +  def getBlacklistTimeout(conf: SparkConf): Long = {
    +    conf.get(config.BLACKLIST_TIMEOUT_CONF).getOrElse {
    +      conf.get(config.BLACKLIST_LEGACY_TIMEOUT_CONF).getOrElse {
    +        Utils.timeStringAsMs(DEFAULT_TIMEOUT)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * 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 */
    +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"
    +  }
    +}
    +
    +/**
    + * 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"
    +  }
    +
    --- End diff --
    
    remove extra blank line


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r86606181
  
    --- Diff: 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])
    --- End diff --
    
    TODO Is this used??


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r88164697
  
    --- Diff: yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala ---
    @@ -121,13 +121,20 @@ private[spark] abstract class YarnSchedulerBackend(
         }
       }
     
    +  private[cluster] def prepareRequestExecutors(requestedTotal: Int): RequestExecutors = {
    --- End diff --
    
    I certainly could use a mock instead -- but I actually thought that refactoring for testing is preferable, and mocks are only used when doing that kind of refactoring is too complex for the task at hand / code you don't control / etc.  (plus some judgement of which is cleaner).  In this case, I think separating the logic from just sending msgs around seems like a good thing.
    
    but don't have a really strong preference.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r72514956
  
    --- 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")
    +        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 (executorIdToBlacklistStatus.nonEmpty) {
    +        nextExpiryTime = executorIdToBlacklistStatus.map{_._2.expiryTime}.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(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
    +        executorIdToBlacklistStatus.put(exec, BlacklistedExecutor(node, expiryTime))
    +        val blacklistedExecsOnNode = nodeToFailedExecs.getOrElseUpdate(node, HashSet[String]())
    +        blacklistedExecsOnNode += exec
    +        executorIdToFailureCount.remove(exec)
    +        if (expiryTime < nextExpiryTime) {
    --- End diff --
    
    When would this evaluate to false? 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r75502668
  
    --- Diff: core/src/main/scala/org/apache/spark/TaskEndReason.scala ---
    @@ -204,6 +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
    --- End diff --
    
    why is this a val and not def?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r72534363
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -280,11 +302,22 @@ private[spark] class TaskSchedulerImpl(
           }
         }
         if (!launchedTask) {
    -      taskSet.abortIfCompletelyBlacklisted(executorIdToHost.keys)
    +      blacklistTracker.foreach(taskSet.abortIfCompletelyBlacklisted(executorsByHost, _))
         }
         return launchedTask
       }
     
    +  private[scheduler] def areAllExecutorsBlacklisted(): Boolean = {
    +    blacklistTracker match {
    --- End diff --
    
    blacklistTracker.map { .. }.getOrElse(false)?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r86600439
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -306,12 +323,24 @@ private[spark] class TaskSchedulerImpl(
           }
         }
     
    +    // 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
    --- End diff --
    
    why'd this move? (it's a little confusing up here since it's not used until later)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/62119/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r77297662
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.BeforeAndAfterEach
    +import org.scalatest.mock.MockitoSugar
    +
    +import org.apache.spark._
    +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 blacklistTracker: BlacklistTracker = _
    +
    +  override def afterEach(): Unit = {
    +    if (blacklistTracker != null) {
    +      blacklistTracker = null
    +    }
    +    super.afterEach()
    +  }
    +
    +  val allOptions = (('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 = {
    +    allOptions.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
    +  }
    +
    +  test("Blacklisting individual tasks") {
    +    val conf = new SparkConf().setAppName("test").setMaster("local")
    +      .set(config.BLACKLIST_ENABLED.key, "true")
    +    val scheduler = mockTaskSchedWithConf(conf)
    +    // 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)
    +    tsm.updateBlacklistForFailedTask("hostA", "1", 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(blacklistTracker.nodeBlacklist() === Set())
    +    assertEquivalentToSet(blacklistTracker.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,
    +    // 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)
    +    // 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 badPart = (index == 0 || index == 1)
    +        val taskExp = (badExec && badPart)
    +        assert(
    +          tsm.isExecutorBlacklistedForTask(executor, index) === taskExp)
    +        val executorExp = badExec
    +        assert(tsm.isExecutorBlacklistedForTaskSet(executor) === executorExp)
    +      }
    +    }
    +    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())
    +
    +    // 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()
    +  }
    +
    +  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)
    +    blacklistTracker = new BlacklistTracker(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 =>
    +      val taskSet = FakeTask.createTaskSet(1)
    +      val tsm = new TaskSetManager(scheduler, Some(tracker), taskSet, 4, clock)
    +      tsm.updateBlacklistForFailedTask("hostA", "1", 0)
    +      tracker.updateBlacklistForSuccessfulTaskSet(stage, 0, tsm.execToFailures)
    +    }
    +    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 =>
    +      val taskSet = FakeTask.createTaskSet(1)
    +      val tsm = new TaskSetManager(scheduler, Some(tracker), taskSet, 4, clock)
    +      tsm.updateBlacklistForFailedTask("hostA", "1", 0)
    +    }
    +    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)
    +      val taskSet = FakeTask.createTaskSet(4, stageId, 0)
    +      val tsm = new TaskSetManager(scheduler, Some(tracker), taskSet, 4, clock)
    +      (0 until 4).foreach { partition =>
    +        tsm.updateBlacklistForFailedTask("hostA", "1", partition)
    +      }
    +      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.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
    +        // stages
    +        assertEquivalentToSet(tracker.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, Some(tracker), taskSet0, 4, clock)
    +    (0 until 4).foreach { partition =>
    +      tsm0.updateBlacklistForFailedTask("hostA", "1", partition)
    +    }
    +    tracker.updateBlacklistForSuccessfulTaskSet(0, 0, tsm0.execToFailures)
    +    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 =>
    +      tsm1.updateBlacklistForFailedTask("hostA", "2", partition)
    +    }
    +    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.applyBlacklistTimeout()
    +    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
    +    val taskSet2 = FakeTask.createTaskSet(4, 2, 0)
    +    val tsm2 = new TaskSetManager(scheduler, Some(tracker), taskSet2, 4, clock)
    +    tsm2.updateBlacklistForFailedTask("hostA", "1", 0)
    +    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
    +    // 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)
    +    // 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)
    +    }
    +    tracker.handleRemovedExecutor("1")
    +    tracker.updateBlacklistForSuccessfulTaskSet(0, 0, tsm0.execToFailures)
    +    assert(tracker.isExecutorBlacklisted("1"))
    +    clock.advance(tracker.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, Some(tracker), taskSet1, 4, clock)
    +    (0 until 4).foreach { partition =>
    +      tsm1.updateBlacklistForFailedTask("hostA", "2", partition)
    +    }
    +    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.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(exec: String): Unit = {
    +      val taskSet = FakeTask.createTaskSet(1, stageId, 0)
    +      val tsm = new TaskSetManager(scheduler, Some(tracker), taskSet, 1, clock)
    +      tsm.updateBlacklistForFailedTask("host-" + exec, exec, 0)
    +      tracker.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)
    +
    +    // We advance the clock past the expiry time.
    +    clock.advance(tracker.BLACKLIST_TIMEOUT_MILLIS + 1)
    +    val t0 = clock.getTimeMillis()
    +    tracker.applyBlacklistTimeout()
    +    assert(tracker.nextExpiryTime === Long.MaxValue)
    +    failOneTaskInTaskSet("1")
    +
    +    // 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.setTime(t0 + tracker.BLACKLIST_TIMEOUT_MILLIS)
    +    val t1 = clock.getTimeMillis()
    +    failOneTaskInTaskSet("1")
    +    tracker.applyBlacklistTimeout()
    +    assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1"))
    +    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)
    +    val t2 = clock.getTimeMillis()
    +    failOneTaskInTaskSet("3")
    +    failOneTaskInTaskSet("3")
    +    tracker.applyBlacklistTimeout()
    +    assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1", "3"))
    +    assert(tracker.nextExpiryTime === t1 + tracker.BLACKLIST_TIMEOUT_MILLIS)
    +
    +
    --- End diff --
    
    remove extra newline here (and below)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r76476924
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -0,0 +1,385 @@
    +/*
    + * 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.{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.
    +   */
    +  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.
    +   */
    +  private 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 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.
    +        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)
    +          }
    +        }
    +      }
    +      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.
    +        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 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 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" +
    +          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.
    +        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
    +  }
    +}
    +
    +
    +private[scheduler] object BlacklistTracker extends Logging {
    +
    +  private val DEFAULT_TIMEOUT = "1h"
    +
    +  /**
    +   * 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:
    +   *   - off for local mode
    +   *   - on for distributed modes (including local-cluster)
    +   */
    +  def isBlacklistEnabled(conf: SparkConf): Boolean = {
    +    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 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
    +            } 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
    +            !Utils.isLocalMaster(conf)
    +        }
    +    }
    +  }
    +
    +  def getBlacklistTimeout(conf: SparkConf): Long = {
    +    conf.get(config.BLACKLIST_TIMEOUT_CONF).getOrElse {
    +      conf.get(config.BLACKLIST_LEGACY_TIMEOUT_CONF).getOrElse {
    +        Utils.timeStringAsMs(DEFAULT_TIMEOUT)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * 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 */
    +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 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"
    +  }
    +}
    +
    +/**
    + * 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
    --- End diff --
    
    ah thanks @tgravescs.  task failure timeouts were pretty messed up -- it was only working if all failures were on nodes that were eventually 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #68790 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/68790/consoleFull)** for PR 14079 at commit [`5d8500a`](https://github.com/apache/spark/commit/5d8500a948ee7a2df5b59bef71bc0a52fc4f04de).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r92103410
  
    --- Diff: yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala ---
    @@ -691,11 +691,11 @@ private[spark] class ApplicationMaster(
         }
     
         override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
    -      case RequestExecutors(requestedTotal, localityAwareTasks, hostToLocalTaskCount) =>
    +      case r: RequestExecutors =>
    --- End diff --
    
    Ok cool seems fine


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #70120 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/70120/consoleFull)** for PR 14079 at commit [`c95462f`](https://github.com/apache/spark/commit/c95462fe5c25d37b8658955304f739cc10ccf1f9).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/63989/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #62737 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62737/consoleFull)** for PR 14079 at commit [`8a12adf`](https://github.com/apache/spark/commit/8a12adf445b00e8841eb3df071c0b6adee6c16da).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r74671218
  
    --- 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")
    +        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 (executorIdToBlacklistStatus.nonEmpty) {
    +        nextExpiryTime = executorIdToBlacklistStatus.map{_._2.expiryTime}.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(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
    +        executorIdToBlacklistStatus.put(exec, BlacklistedExecutor(node, expiryTime))
    +        val blacklistedExecsOnNode = nodeToFailedExecs.getOrElseUpdate(node, HashSet[String]())
    +        blacklistedExecsOnNode += exec
    +        executorIdToFailureCount.remove(exec)
    +        if (expiryTime < nextExpiryTime) {
    +          nextExpiryTime = expiryTime
    +        }
    +
    +        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)
    +      }
    +    }
    +  }
    +
    +  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 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.
    +    executorIdToFailureCount -= executorId
    +  }
    +}
    +
    +
    +private[scheduler] object BlacklistTracker extends Logging {
    +
    +  /**
    +   * 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 = {
    +    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 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) =>
    +              // 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)
    +        }
    +    }
    +  }
    +
    +  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")
    +    }
    +  }
    +}
    +
    +/** 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]()
    --- End diff --
    
    Sounds good!


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #62253 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62253/consoleFull)** for PR 14079 at commit [`fa3e34a`](https://github.com/apache/spark/commit/fa3e34aceb5a20aa7a0324b50d55e71e9a50ae6d).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #65919 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/65919/consoleFull)** for PR 14079 at commit [`0c3ceba`](https://github.com/apache/spark/commit/0c3cebac02eeb194f1792db6235cffd50833a783).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #63993 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/63993/consoleFull)** for PR 14079 at commit [`cec36c9`](https://github.com/apache/spark/commit/cec36c93c16d3b3f02a4720649c187595471663d).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r70004312
  
    --- 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
    +   * the executor is blacklisted overall (or anything to do with the node the executor is on).
    +   */
    +  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)
    +  }
    +
    +  def nodeBlacklist(): Set[String] = {
    +    _nodeBlacklist.get()
    +  }
    +
    +  def isNodeBlacklisted(node: String): Boolean = {
    +    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 previous
    +    // 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.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
    +      val blacklistedExecutors =
    +        stageFailures.filter{_._2.totalFailures >= MAX_FAILURES_PER_EXEC_STAGE}
    --- End diff --
    
    nit: style. use either '.blah(_.something)` or `.blah { o => o.something }`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r71232039
  
    --- Diff: docs/configuration.md ---
    @@ -1149,6 +1149,20 @@ Apart from these, the following properties are also available, and may be useful
       </td>
     </tr>
     <tr>
    +  <td><code>spark.scheduler.blacklist.enabled</code></td>
    +  <td>true</td>
    +  <td>
    +    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.
    +  </td>
    +</tr>
    +<tr>
    +  <td><code>spark.scheduler.blacklist.MORE</code></td>
    +  <td>TODO</td>
    --- End diff --
    
    fill in


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #64220 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/64220/consoleFull)** for PR 14079 at commit [`f8b1bff`](https://github.com/apache/spark/commit/f8b1bffee588df45809519436983cb95c6a481f3).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r72541285
  
    --- 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()
    --- End diff --
    
    maybe just nodeToBlacklistExpiryTime? I think the key here is a hostname, so ID is slightly misleading (since other IDs we use are 0/1/2/3/4 etc.)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r92275871
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala ---
    @@ -408,6 +411,96 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
           }
           assert(tsm.isZombie)
         }
    +
    +    // the tasksSets complete, so the tracker should be notified of the successful ones
    +    verify(blacklist, times(1)).updateBlacklistForSuccessfulTaskSet(
    +      stageId = 0,
    +      stageAttemptId = 0,
    +      failuresByExec = stageToMockTaskSetBlacklist(0).execToFailures)
    +    verify(blacklist, times(1)).updateBlacklistForSuccessfulTaskSet(
    +      stageId = 1,
    +      stageAttemptId = 0,
    +      failuresByExec = stageToMockTaskSetBlacklist(1).execToFailures)
    +    // but we shouldn't update for the failed taskset
    +    verify(blacklist, never).updateBlacklistForSuccessfulTaskSet(
    +      stageId = meq(2),
    +      stageAttemptId = anyInt(),
    +      failuresByExec = anyObject())
    +  }
    +
    +  test("scheduled tasks obey node and executor blacklists") {
    --- End diff --
    
    the point wasn't really to make sure the tsm gets aborted, but it seemed simplest to have all the resources blacklisted.  But I can change it to add one more offered resource, which lets things get scheduled and / or just remove the assert that they get aborted.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #70194 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/70194/testReport)** for PR 14079 at commit [`f249b00`](https://github.com/apache/spark/commit/f249b00e3eb64bf35ab836fa4b89eb961a9511a5).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by squito <gi...@git.apache.org>.
Github user squito commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    @tgravescs not totally sure I know what you mean -- are you saying that with blacklisting and dynamic allocation, you might have a situation with only one executor, which then gets blacklisted, and the app fails rather than creating another executor?  That sounds related to https://issues.apache.org/jira/browse/SPARK-15815.  I agree this isn't ideal, but I can live with it, as long as this only happens when there really is something getting blacklisted.  You haven't encountered scenarios where the blacklist causes an app fail with perfectly good executors, have you?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/69263/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by squito <gi...@git.apache.org>.
Github user squito commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    @kayousterhout @tgravescs sorry for the long delay from me.  I've addressed most the feedback.  But I haven't looked at separating out the blacklist logic into a separate class inside TaskSetManager -- I will look at that and ping you again when I make that update, just thought it might be worth looking at now.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r86609225
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala ---
    @@ -17,10 +17,299 @@
     
     package org.apache.spark.scheduler
     
    -import org.apache.spark.{SparkConf, SparkFunSuite}
    +import org.mockito.Mockito.when
    +import org.scalatest.BeforeAndAfterEach
    +import org.scalatest.mock.MockitoSugar
    +
    +import org.apache.spark._
     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
    +  }
     
    -class BlacklistTrackerSuite extends SparkFunSuite {
    +  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.
    --- End diff --
    
    don't --> shouldn't


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r86648349
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala ---
    @@ -17,10 +17,299 @@
     
     package org.apache.spark.scheduler
     
    -import org.apache.spark.{SparkConf, SparkFunSuite}
    +import org.mockito.Mockito.when
    +import org.scalatest.BeforeAndAfterEach
    +import org.scalatest.mock.MockitoSugar
    +
    +import org.apache.spark._
     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
    +  }
     
    -class BlacklistTrackerSuite extends SparkFunSuite {
    +  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.nodeToBlacklistedExecs("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") {
    --- End diff --
    
    this name is a little misleading -- how about something like
    
    only blacklist nodes for the application when enough executors have failed on that specific host


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #62351 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62351/consoleFull)** for PR 14079 at commit [`7aff08a`](https://github.com/apache/spark/commit/7aff08a52099334eb6ac242c7ba70a9873aef624).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r70007341
  
    --- 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
    +   * the executor is blacklisted overall (or anything to do with the node the executor is on).
    +   */
    +  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)
    +  }
    +
    +  def nodeBlacklist(): Set[String] = {
    +    _nodeBlacklist.get()
    +  }
    +
    +  def isNodeBlacklisted(node: String): Boolean = {
    +    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 previous
    +    // 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.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
    +      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.
    +   */
    +  def isExecutorBlacklisted(
    +      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
    +    }
    +  }
    +
    +  def isNodeBlacklisted(
    +      node: String,
    +      stageId: Int,
    +      indexInTaskSet: Int): Boolean = {
    +    stageIdToNodeBlacklistedTasks.get(stageId).flatMap { nodeToFailures =>
    +      nodeToFailures.get(node).map{_.contains(indexInTaskSet)}
    +    }.getOrElse(false)
    +  }
    +
    +  def removeExecutor(executorId: String): Unit = {
    +    executorIdToBlacklistExpiryTime -= executorId
    +    executorIdToFailureCount -= executorId
    +    stageIdToExecToFailures.values.foreach { execFailureOneStage =>
    +      execFailureOneStage -= executorId
    +    }
    +  }
    +}
    +
    +
    +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:
    +   * 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)
    --- End diff --
    
    If I understand this correctly, if a user has explicitly set `spark.scheduler.executorTaskBlacklistTime=0` in the configuration, that will fail to disable the blacklist, right?
    
    Maybe you should be using `getOption` here to account for that case?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r76682765
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -0,0 +1,395 @@
    +/*
    + * 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.{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[scheduler] 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.
    +   */
    +  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
    +   * 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
    +  }
    +}
    +
    +
    +private[scheduler] object BlacklistTracker extends Logging {
    +
    +  private val DEFAULT_TIMEOUT = "1h"
    +
    +  /**
    +   * 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:
    +   *   - off for local mode
    +   *   - on for distributed modes (including local-cluster)
    +   */
    +  def isBlacklistEnabled(conf: SparkConf): Boolean = {
    +    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 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
    +            } 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
    +            !Utils.isLocalMaster(conf)
    +        }
    +    }
    +  }
    +
    +  def getBlacklistTimeout(conf: SparkConf): Long = {
    +    conf.get(config.BLACKLIST_TIMEOUT_CONF).getOrElse {
    +      conf.get(config.BLACKLIST_LEGACY_TIMEOUT_CONF).getOrElse {
    +        Utils.timeStringAsMs(DEFAULT_TIMEOUT)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * 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 " +
    --- End diff --
    
    should be >=


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r77297969
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.BeforeAndAfterEach
    +import org.scalatest.mock.MockitoSugar
    +
    +import org.apache.spark._
    +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 blacklistTracker: BlacklistTracker = _
    +
    +  override def afterEach(): Unit = {
    +    if (blacklistTracker != null) {
    +      blacklistTracker = null
    +    }
    +    super.afterEach()
    +  }
    +
    +  val allOptions = (('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 = {
    +    allOptions.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
    +  }
    +
    +  test("Blacklisting individual tasks") {
    +    val conf = new SparkConf().setAppName("test").setMaster("local")
    +      .set(config.BLACKLIST_ENABLED.key, "true")
    +    val scheduler = mockTaskSchedWithConf(conf)
    +    // 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)
    +    tsm.updateBlacklistForFailedTask("hostA", "1", 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(blacklistTracker.nodeBlacklist() === Set())
    +    assertEquivalentToSet(blacklistTracker.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,
    +    // 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)
    +    // 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 badPart = (index == 0 || index == 1)
    +        val taskExp = (badExec && badPart)
    +        assert(
    +          tsm.isExecutorBlacklistedForTask(executor, index) === taskExp)
    +        val executorExp = badExec
    +        assert(tsm.isExecutorBlacklistedForTaskSet(executor) === executorExp)
    +      }
    +    }
    +    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())
    +
    +    // 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()
    +  }
    +
    +  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)
    +    blacklistTracker = new BlacklistTracker(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 =>
    +      val taskSet = FakeTask.createTaskSet(1)
    +      val tsm = new TaskSetManager(scheduler, Some(tracker), taskSet, 4, clock)
    +      tsm.updateBlacklistForFailedTask("hostA", "1", 0)
    +      tracker.updateBlacklistForSuccessfulTaskSet(stage, 0, tsm.execToFailures)
    +    }
    +    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 =>
    +      val taskSet = FakeTask.createTaskSet(1)
    +      val tsm = new TaskSetManager(scheduler, Some(tracker), taskSet, 4, clock)
    +      tsm.updateBlacklistForFailedTask("hostA", "1", 0)
    +    }
    +    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)
    +      val taskSet = FakeTask.createTaskSet(4, stageId, 0)
    +      val tsm = new TaskSetManager(scheduler, Some(tracker), taskSet, 4, clock)
    +      (0 until 4).foreach { partition =>
    +        tsm.updateBlacklistForFailedTask("hostA", "1", partition)
    +      }
    +      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.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
    +        // stages
    +        assertEquivalentToSet(tracker.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, Some(tracker), taskSet0, 4, clock)
    +    (0 until 4).foreach { partition =>
    +      tsm0.updateBlacklistForFailedTask("hostA", "1", partition)
    +    }
    +    tracker.updateBlacklistForSuccessfulTaskSet(0, 0, tsm0.execToFailures)
    +    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 =>
    +      tsm1.updateBlacklistForFailedTask("hostA", "2", partition)
    +    }
    +    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.applyBlacklistTimeout()
    +    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
    +    val taskSet2 = FakeTask.createTaskSet(4, 2, 0)
    +    val tsm2 = new TaskSetManager(scheduler, Some(tracker), taskSet2, 4, clock)
    +    tsm2.updateBlacklistForFailedTask("hostA", "1", 0)
    +    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
    +    // 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)
    +    // 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)
    +    }
    +    tracker.handleRemovedExecutor("1")
    +    tracker.updateBlacklistForSuccessfulTaskSet(0, 0, tsm0.execToFailures)
    +    assert(tracker.isExecutorBlacklisted("1"))
    +    clock.advance(tracker.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, Some(tracker), taskSet1, 4, clock)
    +    (0 until 4).foreach { partition =>
    +      tsm1.updateBlacklistForFailedTask("hostA", "2", partition)
    +    }
    +    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.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(exec: String): Unit = {
    +      val taskSet = FakeTask.createTaskSet(1, stageId, 0)
    +      val tsm = new TaskSetManager(scheduler, Some(tracker), taskSet, 1, clock)
    +      tsm.updateBlacklistForFailedTask("host-" + exec, exec, 0)
    +      tracker.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)
    +
    +    // We advance the clock past the expiry time.
    +    clock.advance(tracker.BLACKLIST_TIMEOUT_MILLIS + 1)
    +    val t0 = clock.getTimeMillis()
    +    tracker.applyBlacklistTimeout()
    +    assert(tracker.nextExpiryTime === Long.MaxValue)
    +    failOneTaskInTaskSet("1")
    +
    +    // 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.setTime(t0 + tracker.BLACKLIST_TIMEOUT_MILLIS)
    --- End diff --
    
    maybe throw in a -1 here? this seems brittle to a future > being changed to >=


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #65922 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/65922/consoleFull)** for PR 14079 at commit [`2381b25`](https://github.com/apache/spark/commit/2381b2526cc2654258a85d496d9416ded06eeb20).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r79933045
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.BeforeAndAfterEach
    +import org.scalatest.mock.MockitoSugar
    +
    +import org.apache.spark._
    +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 blacklistTracker: BlacklistTracker = _
    +
    +  override def afterEach(): Unit = {
    +    if (blacklistTracker != null) {
    +      blacklistTracker = null
    +    }
    +    super.afterEach()
    +  }
    +
    +  val allOptions = (('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 = {
    +    allOptions.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
    +  }
    +
    +  test("Blacklisting individual tasks") {
    +    val conf = new SparkConf().setAppName("test").setMaster("local")
    +      .set(config.BLACKLIST_ENABLED.key, "true")
    +    val scheduler = mockTaskSchedWithConf(conf)
    +    // 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)
    +    tsm.updateBlacklistForFailedTask("hostA", "1", 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(blacklistTracker.nodeBlacklist() === Set())
    +    assertEquivalentToSet(blacklistTracker.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,
    +    // 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)
    +    // 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 badPart = (index == 0 || index == 1)
    +        val taskExp = (badExec && badPart)
    +        assert(
    +          tsm.isExecutorBlacklistedForTask(executor, index) === taskExp)
    +        val executorExp = badExec
    +        assert(tsm.isExecutorBlacklistedForTaskSet(executor) === executorExp)
    +      }
    +    }
    +    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())
    +
    +    // when the stage completes successfully, now there is sufficient evidence we've got
    +    // bad executors and node
    +    blacklistTracker.updateBlacklistForSuccessfulTaskSet(0, 0, tsm.execToFailures)
    --- End diff --
    
    the test isn't bothering to explicit mark all of the tasks as succeeded -- trying to find a balance between recreating a ton of scheduler behavior within the test vs directly calling internals bits.  I could change this if you want -- but basically it would still be this line getting converted into a call to a utility function like `finishRemainingTasksSuccessfully(tsm)`.
    
    FWIW, the scheduler integration tests do a better job of this -- by testing at a much higher level, you avoid having to recreate behavior within the test itself.  It also gets to your point about perhaps extracting out the blacklisting logic from tsm into a helper -- really here we're just testing interaction with that helper, not with the entire tsm. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r79936008
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.BeforeAndAfterEach
    +import org.scalatest.mock.MockitoSugar
    +
    +import org.apache.spark._
    +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 blacklistTracker: BlacklistTracker = _
    +
    +  override def afterEach(): Unit = {
    +    if (blacklistTracker != null) {
    +      blacklistTracker = null
    +    }
    +    super.afterEach()
    +  }
    +
    +  val allOptions = (('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 = {
    +    allOptions.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
    +  }
    +
    +  test("Blacklisting individual tasks") {
    +    val conf = new SparkConf().setAppName("test").setMaster("local")
    +      .set(config.BLACKLIST_ENABLED.key, "true")
    +    val scheduler = mockTaskSchedWithConf(conf)
    +    // 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)
    +    tsm.updateBlacklistForFailedTask("hostA", "1", 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(blacklistTracker.nodeBlacklist() === Set())
    +    assertEquivalentToSet(blacklistTracker.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,
    +    // 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)
    +    // 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 badPart = (index == 0 || index == 1)
    +        val taskExp = (badExec && badPart)
    +        assert(
    +          tsm.isExecutorBlacklistedForTask(executor, index) === taskExp)
    +        val executorExp = badExec
    +        assert(tsm.isExecutorBlacklistedForTaskSet(executor) === executorExp)
    +      }
    +    }
    +    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())
    +
    +    // 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()
    +  }
    +
    +  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)
    +    blacklistTracker = new BlacklistTracker(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 =>
    +      val taskSet = FakeTask.createTaskSet(1)
    +      val tsm = new TaskSetManager(scheduler, Some(tracker), taskSet, 4, clock)
    +      tsm.updateBlacklistForFailedTask("hostA", "1", 0)
    +      tracker.updateBlacklistForSuccessfulTaskSet(stage, 0, tsm.execToFailures)
    +    }
    +    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 =>
    +      val taskSet = FakeTask.createTaskSet(1)
    +      val tsm = new TaskSetManager(scheduler, Some(tracker), taskSet, 4, clock)
    +      tsm.updateBlacklistForFailedTask("hostA", "1", 0)
    +    }
    +    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)
    +      val taskSet = FakeTask.createTaskSet(4, stageId, 0)
    +      val tsm = new TaskSetManager(scheduler, Some(tracker), taskSet, 4, clock)
    +      (0 until 4).foreach { partition =>
    +        tsm.updateBlacklistForFailedTask("hostA", "1", partition)
    +      }
    +      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.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
    +        // stages
    +        assertEquivalentToSet(tracker.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, Some(tracker), taskSet0, 4, clock)
    +    (0 until 4).foreach { partition =>
    +      tsm0.updateBlacklistForFailedTask("hostA", "1", partition)
    +    }
    +    tracker.updateBlacklistForSuccessfulTaskSet(0, 0, tsm0.execToFailures)
    +    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 =>
    +      tsm1.updateBlacklistForFailedTask("hostA", "2", partition)
    +    }
    +    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.applyBlacklistTimeout()
    +    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
    +    val taskSet2 = FakeTask.createTaskSet(4, 2, 0)
    +    val tsm2 = new TaskSetManager(scheduler, Some(tracker), taskSet2, 4, clock)
    +    tsm2.updateBlacklistForFailedTask("hostA", "1", 0)
    +    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
    +    // 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)
    +    // 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)
    +    }
    +    tracker.handleRemovedExecutor("1")
    +    tracker.updateBlacklistForSuccessfulTaskSet(0, 0, tsm0.execToFailures)
    +    assert(tracker.isExecutorBlacklisted("1"))
    +    clock.advance(tracker.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, Some(tracker), taskSet1, 4, clock)
    +    (0 until 4).foreach { partition =>
    +      tsm1.updateBlacklistForFailedTask("hostA", "2", partition)
    +    }
    +    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.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(exec: String): Unit = {
    +      val taskSet = FakeTask.createTaskSet(1, stageId, 0)
    +      val tsm = new TaskSetManager(scheduler, Some(tracker), taskSet, 1, clock)
    +      tsm.updateBlacklistForFailedTask("host-" + exec, exec, 0)
    +      tracker.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)
    +
    +    // We advance the clock past the expiry time.
    +    clock.advance(tracker.BLACKLIST_TIMEOUT_MILLIS + 1)
    +    val t0 = clock.getTimeMillis()
    +    tracker.applyBlacklistTimeout()
    +    assert(tracker.nextExpiryTime === Long.MaxValue)
    +    failOneTaskInTaskSet("1")
    +
    +    // 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.setTime(t0 + tracker.BLACKLIST_TIMEOUT_MILLIS)
    +    val t1 = clock.getTimeMillis()
    +    failOneTaskInTaskSet("1")
    +    tracker.applyBlacklistTimeout()
    +    assertEquivalentToSet(tracker.isExecutorBlacklisted(_), Set("1"))
    +    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)
    +    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)
    +
    +
    +    // 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())
    +    // 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)
    --- End diff --
    
    yes in a way, but it seems worth checking again, since the earlier check is more of an initial condition + no update, while this is checking that it is getting reset correctly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by squito <gi...@git.apache.org>.
Github user squito commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Jenkins, retest this please


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r92079769
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala ---
    @@ -17,10 +17,322 @@
     
     package org.apache.spark.scheduler
     
    -import org.apache.spark.{SparkConf, SparkFunSuite}
    +import org.mockito.Mockito.when
    +import org.scalatest.BeforeAndAfterEach
    +import org.scalatest.mock.MockitoSugar
    +
    +import org.apache.spark._
     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 beforeEach(): Unit = {
    +    conf = new SparkConf().setAppName("test").setMaster("local")
    +      .set(config.BLACKLIST_ENABLED.key, "true")
    +    scheduler = mockTaskSchedWithConf(conf)
    +
    +    clock.setTime(0)
    +    blacklist = new BlacklistTracker(conf, clock)
    +  }
    +
    +  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, 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 { id =>
    +      val actual = f(id)
    +      val exp = expected.contains(id)
    +      assert(actual === exp, raw"""for string "$id" """)
    +    }
    +  }
    +
    +  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)
    +  }
    +
    +  test("executors can be blacklisted with only a few failures per stage") {
    +    // For 4 different stages, executor 1 fails a task, then executor 2 succeeds the task,
    --- End diff --
    
    I don't think this is for 4 different stages? (looks like ~20?)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/62630/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r72540361
  
    --- Diff: yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala ---
    @@ -217,18 +219,34 @@ 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.
    --- End diff --
    
    Containers


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r86606779
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala ---
    @@ -17,10 +17,299 @@
     
     package org.apache.spark.scheduler
     
    -import org.apache.spark.{SparkConf, SparkFunSuite}
    +import org.mockito.Mockito.when
    +import org.scalatest.BeforeAndAfterEach
    +import org.scalatest.mock.MockitoSugar
    +
    +import org.apache.spark._
     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 =>
    --- End diff --
    
    opt is a little confusing here, since it's not an option? what about id instead?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/70102/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #69433 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/69433/consoleFull)** for PR 14079 at commit [`35978e2`](https://github.com/apache/spark/commit/35978e28c78ddac67613d01f6653cf8b3451b42a).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #68694 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/68694/consoleFull)** for PR 14079 at commit [`cc3faaf`](https://github.com/apache/spark/commit/cc3faaf720148e2c92bb6858231929473b5ab19f).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/65783/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r72537531
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala ---
    @@ -266,19 +278,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
    -    }
    -
    -    false
    +  private def blacklistedOnExec(execId: String, host: String, index: Int): Boolean = {
    +    blacklistTracker.map { bl =>
    --- End diff --
    
    you can use _ since b1 is never used


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Can you update the PR description and title? (to eliminate the per-task stuff / make it obvious what's new here)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r86612423
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala ---
    @@ -17,10 +17,299 @@
     
     package org.apache.spark.scheduler
     
    -import org.apache.spark.{SparkConf, SparkFunSuite}
    +import org.mockito.Mockito.when
    +import org.scalatest.BeforeAndAfterEach
    +import org.scalatest.mock.MockitoSugar
    +
    +import org.apache.spark._
     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
    +  }
     
    -class BlacklistTrackerSuite extends SparkFunSuite {
    +  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)
    --- End diff --
    
    can you use named parameters for the 0s


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #67620 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/67620/consoleFull)** for PR 14079 at commit [`cdd9f33`](https://github.com/apache/spark/commit/cdd9f336c11572dd104b3e762defb9a257e9655f).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r72534813
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -236,29 +245,42 @@ 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) {
             taskSetsByStageIdAndAttempt -= manager.taskSet.stageId
           }
         }
         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.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}")
    +    }
       }
     
       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 execId = offer.executorId
    +      val offerBlacklisted = blacklistTracker.map { bl =>
    +        taskSet.isNodeBlacklistedForTaskSet(host) ||
    +          taskSet.isExecutorBlacklistedForTaskSet(execId)
    +      }.getOrElse(false)
    +      if (!offerBlacklisted && availableCpus(i) >= CPUS_PER_TASK) {
    --- End diff --
    
    It seems more intuitive to me to move all of the taskset-specific blacklisting to the TaskSetManager, since in general that class encompasses all of the taskset-specific logic.  E.g., taskSet.abortIfCompletelyBlacklisted could itself check with the blacklist tracker.  That would also allow the TaskSetMAnager to hide some complexity, like checking if the executor is blacklisted for the task set.
    
    Calling taskSetSucceeded could also move to the TaskSetManager, so that "success" didn't need to be passed into taskSetFinished.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r90543965
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -17,10 +17,275 @@
     
     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.Utils
    +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 = new  HashMap[String, ExecutorFailureList]()
    +  val executorIdToBlacklistStatus = new HashMap[String, BlacklistedExecutor]()
    +  val nodeIdToBlacklistExpiryTime = new HashMap[String, Long]()
    +  /**
    +   * An immutable copy of the set of nodes that are currently blacklisted.  Kept in an
    +   * AtomicReference to make [[nodeBlacklist()]] thread-safe.
    +   */
    +  private val _nodeBlacklist = new AtomicReference[Set[String]](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 clean up the list of blacklisted executors once an executor has
    +   * been blacklisted for BLACKLIST_TIMEOUT_MILLIS.
    +   */
    +  val nodeToBlacklistedExecs = new HashMap[String, HashSet[String]]()
    +
    +  /**
    +   * Un-blacklists executors and nodes that have been blacklisted for at least
    +   * BLACKLIST_TIMEOUT_MILLIS
    +   */
    +  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"for those executors has timed out")
    +        execsToUnblacklist.foreach { exec =>
    +          val status = executorIdToBlacklistStatus.remove(exec).get
    +          val failedExecsOnNode = nodeToBlacklistedExecs(status.node)
    +          failedExecsOnNode.remove(exec)
    +          if (failedExecsOnNode.isEmpty) {
    +            nodeToBlacklistedExecs.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")
    +        nodeIdToBlacklistExpiryTime --= nodesToUnblacklist
    +        _nodeBlacklist.set(nodeIdToBlacklistExpiryTime.keySet.toSet)
    +      }
    +      updateNextExpiryTime()
    +    }
    +  }
    +
    +  private def updateNextExpiryTime(): Unit = {
    +    val execMinExpiry = if (executorIdToBlacklistStatus.nonEmpty) {
    +      executorIdToBlacklistStatus.map{_._2.expiryTime}.min
    +    } else {
    +      Long.MaxValue
    +    }
    +    val nodeMinExpiry = if (nodeIdToBlacklistExpiryTime.nonEmpty) {
    +      nodeIdToBlacklistExpiryTime.values.min
    +    } else {
    +      Long.MaxValue
    +    }
    +    nextExpiryTime = math.min(execMinExpiry, nodeMinExpiry)
    +  }
    +
    +
    +  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 =
    --- End diff --
    
    naming nit: what about "appFailuresOnExecutor"? (to make it clear that this is for the whole application, as opposed to the whole task set)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/62134/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/62245/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r77274151
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -0,0 +1,393 @@
    +/*
    + * 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.{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[scheduler] 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.
    +   */
    +  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
    +   * 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
    --- End diff --
    
    Can you add a comment about why this doesn't need to look at nodeIdToBlacklistExpiryTime?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    correct, just not ideal right now but can live with it. I haven't seen it cause failure without task failures.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r86473290
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/ExecutorFailuresInTaskSet.scala ---
    @@ -25,26 +25,30 @@ 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.
    +   * executor and the expiry time.
        */
    -  val taskToFailureCount = HashMap[Int, Int]()
    +  val taskToFailureCountAndExpiryTime = HashMap[Int, (Int, Long)]()
    --- End diff --
    
    Given that the expiry time is only used in BlacklistTracker, I think it would be better to store the failure time here -- so all of the logic of handling the expiration time can be encapsulated in the BlacklistTracker (and that also makes it MARGINALLY more clear that the task-set isn't actually doing any expiration)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    I got through everything except the tests.  Can you fix the merge conflicts (which I suspect relate to Josh's change)?
    
    One more meta comment: there's a ton of blacklisting logic, especially in the TSM.  Do you think it would make sense to have a second TSM-specific blacklist helper class that encapsulates all of this logic, and have the TSM call methods from that class? It seems like the blacklisting data structures separate nicely, and it might help the TSM to be more readable.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r86473130
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -17,10 +17,254 @@
     
     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.Utils
    +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 = new  HashMap[String, ExecutorFailureList]()
    +  val executorIdToBlacklistStatus = new HashMap[String, BlacklistedExecutor]()
    +  val nodeIdToBlacklistExpiryTime = new HashMap[String, Long]()
    +  /**
    +   * An immutable copy of the set of nodes that are currently blacklisted.  Kept in an
    +   * AtomicReference to make [[nodeBlacklist()]] thread-safe.
    +   */
    +  private val _nodeBlacklist = new AtomicReference[Set[String]](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 nodeToBlacklistedExecs = new HashMap[String, HashSet[String]]()
    +
    +  /**
    +   * Un-blacklists executors and nodes that have been blacklisted for at least
    +   * BLACKLIST_TIMEOUT_MILLIS
    +   */
    +  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"for those executors has timed out")
    +        execsToUnblacklist.foreach { exec =>
    +          val status = executorIdToBlacklistStatus.remove(exec).get
    +          val failedExecsOnNode = nodeToBlacklistedExecs(status.node)
    +          failedExecsOnNode.remove(exec)
    +          if (failedExecsOnNode.isEmpty) {
    +            nodeToBlacklistedExecs.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 = {
    +    // we don't need to check nodeIdToBlacklistExpiryTime because that will always share an
    +    // expiry time with some blacklisted executor.  In other words, the next node expiry time
    +    // will never be before the next executor blacklist time.
    +    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 = nodeToBlacklistedExecs.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))
    --- End diff --
    
    would it make sense to store the TaskId in taskToFailureCountAndExpiryTime rather than storing the index, since that's what's eventually needed anyway, and then maybe an existing TaskId object can be re-used?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r86466456
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -17,10 +17,254 @@
     
     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.Utils
    +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 = new  HashMap[String, ExecutorFailureList]()
    +  val executorIdToBlacklistStatus = new HashMap[String, BlacklistedExecutor]()
    +  val nodeIdToBlacklistExpiryTime = new HashMap[String, Long]()
    +  /**
    +   * An immutable copy of the set of nodes that are currently blacklisted.  Kept in an
    +   * AtomicReference to make [[nodeBlacklist()]] thread-safe.
    +   */
    +  private val _nodeBlacklist = new AtomicReference[Set[String]](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.
    --- End diff --
    
    change the end to "and we clean up the list of blacklisted executors once an executor has been blacklisted for BLACKLIST_TIMEOUT_MILLIS"? (o/w not obvious what "periodically clean up" means)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by squito <gi...@git.apache.org>.
Github user squito commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    I took another look at having BlacklistTracker just be an option, rather than having a NoopBlacklist.  After some other cleanup, I decided it made more sense to go back to the option, but its in one commit so easy to go either way https://github.com/apache/spark/pull/14079/commits/a34e9aeb695958c749d306595d1adebe0207fdf9


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #64498 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/64498/consoleFull)** for PR 14079 at commit [`cc3b968`](https://github.com/apache/spark/commit/cc3b9683940eebc930e9bf7a5088dc987da23595).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r77276392
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -0,0 +1,393 @@
    +/*
    + * 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.{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[scheduler] 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.
    +   */
    +  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
    +   * 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
    +  }
    +}
    +
    +
    +private[scheduler] object BlacklistTracker extends Logging {
    +
    +  private val DEFAULT_TIMEOUT = "1h"
    +
    +  /**
    +   * 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:
    +   *   - off for local mode
    +   *   - on for distributed modes (including local-cluster)
    +   */
    +  def isBlacklistEnabled(conf: SparkConf): Boolean = {
    +    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 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
    +            } 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
    +            !Utils.isLocalMaster(conf)
    +        }
    +    }
    +  }
    +
    +  def getBlacklistTimeout(conf: SparkConf): Long = {
    +    conf.get(config.BLACKLIST_TIMEOUT_CONF).getOrElse {
    +      conf.get(config.BLACKLIST_LEGACY_TIMEOUT_CONF).getOrElse {
    +        Utils.timeStringAsMs(DEFAULT_TIMEOUT)
    +      }
    +    }
    +  }
    +
    +  /**
    +   * 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 (maxNodeAttempts >= maxTaskFailures) {
    +        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}")
    +      }
    +    }
    +
    +  }
    +}
    +
    +/** 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"
    +  }
    +}
    +
    +/**
    + * 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)
    --- End diff --
    
    move this class inside BlacklistTracker?
    
    Or I wonder if it would be better to rename the map to executorIdToNodeAndExpiryTime and then just put a 2-item tuple in the map rather than this simple datastructure (don't have strong feelings though, if you prefer the class)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r72525614
  
    --- 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")
    +        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 (executorIdToBlacklistStatus.nonEmpty) {
    +        nextExpiryTime = executorIdToBlacklistStatus.map{_._2.expiryTime}.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(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
    +        executorIdToBlacklistStatus.put(exec, BlacklistedExecutor(node, expiryTime))
    +        val blacklistedExecsOnNode = nodeToFailedExecs.getOrElseUpdate(node, HashSet[String]())
    +        blacklistedExecsOnNode += exec
    +        executorIdToFailureCount.remove(exec)
    +        if (expiryTime < nextExpiryTime) {
    +          nextExpiryTime = expiryTime
    +        }
    +
    +        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)
    +      }
    +    }
    +  }
    +
    +  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 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.
    +    executorIdToFailureCount -= executorId
    +  }
    +}
    +
    +
    +private[scheduler] object BlacklistTracker extends Logging {
    +
    +  /**
    +   * 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 = {
    +    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 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) =>
    +              // 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)
    +        }
    +    }
    +  }
    +
    +  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 {
    --- End diff --
    
    This is also a little hard to read -- did you consider 
    
    conf.get(config.BLACKLIST_EXPIRY_TIMEOUT_CONF).getOrElse(
      conf.get(config.BLACKLIST_LEGACY_TIMEOUT).getOrElse(Utils.timeStringAsMs("1h"))
    



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #65939 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/65939/consoleFull)** for PR 14079 at commit [`278fff3`](https://github.com/apache/spark/commit/278fff343eaa1b917f17d7591e39b0543538d253).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r84149709
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -17,10 +17,254 @@
     
     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.Utils
    +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 = new  HashMap[String, ExecutorFailureList]()
    +  val executorIdToBlacklistStatus = new HashMap[String, BlacklistedExecutor]()
    +  val nodeIdToBlacklistExpiryTime = new HashMap[String, Long]()
    +  /**
    +   * An immutable copy of the set of nodes that are currently blacklisted.  Kept in an
    +   * AtomicReference to make [[nodeBlacklist()]] thread-safe.
    +   */
    +  private val _nodeBlacklist = new AtomicReference[Set[String]](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()
    --- End diff --
    
    The comment about this not growing too large isn't totally correct.  With this PR, we only prevent adding executors on blacklisted nodes in yarn mode.  Standalone and mesos could keep allocating executors on the bad node.
    
    This is particularly a problem with dynamic allocation.  Given the default dynamic allocation timeout of 1 minute, and the blacklist timeout of 1 hour, this means that during that hour, you'll repeatedly spin up executors, blacklist them, and then kill them due to the 1 minute timeout.
    
    Not only do we build up `60 * numExecutorsPerNode`  entries in the inner hashset during that hour -- we also won't ever clean it up, since we only do that when all the executors have hit their timeout.
    
    I haven't come up with a great solution, other than blacklisting nodes in standalone and mesos.  But I'm a little wary of adding that, particularly on mesos, which I don't even know how to test ...


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #69263 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/69263/consoleFull)** for PR 14079 at commit [`72036f4`](https://github.com/apache/spark/commit/72036f4b79b7facd0b300e4f8360dc84c0bb98f3).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r72160497
  
    --- Diff: core/src/main/scala/org/apache/spark/TaskEndReason.scala ---
    @@ -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
    --- End diff --
    
    When is this true?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r70008649
  
    --- 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 =
    --- End diff --
    
    I generally ask people to add new configs to `core/src/main/scala/org/apache/spark/internal/config/package.scala`, but no big deal either way.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #63992 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/63992/consoleFull)** for PR 14079 at commit [`b582d8e`](https://github.com/apache/spark/commit/b582d8e84fc17d908b43b91ad819acfc5b4b79fc).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r77273111
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -0,0 +1,393 @@
    +/*
    + * 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.{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[scheduler] 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.
    +   */
    +  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
    +   * 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 = {
    --- End diff --
    
    Can you add a docstring here? "Un-blacklists executors and nodes that have been blacklisted for at least BLACKLIST_TIMEOUT"?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r77296487
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.BeforeAndAfterEach
    +import org.scalatest.mock.MockitoSugar
    +
    +import org.apache.spark._
    +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 blacklistTracker: BlacklistTracker = _
    +
    +  override def afterEach(): Unit = {
    +    if (blacklistTracker != null) {
    +      blacklistTracker = null
    +    }
    +    super.afterEach()
    +  }
    +
    +  val allOptions = (('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 = {
    +    allOptions.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
    +  }
    +
    +  test("Blacklisting individual tasks") {
    +    val conf = new SparkConf().setAppName("test").setMaster("local")
    +      .set(config.BLACKLIST_ENABLED.key, "true")
    +    val scheduler = mockTaskSchedWithConf(conf)
    +    // 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)
    +    tsm.updateBlacklistForFailedTask("hostA", "1", 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(blacklistTracker.nodeBlacklist() === Set())
    +    assertEquivalentToSet(blacklistTracker.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,
    +    // 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)
    +    // 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 badPart = (index == 0 || index == 1)
    +        val taskExp = (badExec && badPart)
    +        assert(
    +          tsm.isExecutorBlacklistedForTask(executor, index) === taskExp)
    +        val executorExp = badExec
    +        assert(tsm.isExecutorBlacklistedForTaskSet(executor) === executorExp)
    +      }
    +    }
    +    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())
    +
    +    // 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()
    --- End diff --
    
    Why do you need this function?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #65919 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/65919/consoleFull)** for PR 14079 at commit [`0c3ceba`](https://github.com/apache/spark/commit/0c3cebac02eeb194f1792db6235cffd50833a783).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r86611783
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala ---
    @@ -17,10 +17,299 @@
     
     package org.apache.spark.scheduler
     
    -import org.apache.spark.{SparkConf, SparkFunSuite}
    +import org.mockito.Mockito.when
    +import org.scalatest.BeforeAndAfterEach
    +import org.scalatest.mock.MockitoSugar
    +
    +import org.apache.spark._
     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
    +  }
     
    -class BlacklistTrackerSuite extends SparkFunSuite {
    +  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)
    --- End diff --
    
    // Advance the clock and then make sure hostA and executors 1 and 2 have been removed from the blacklist


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r79920803
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.BeforeAndAfterEach
    +import org.scalatest.mock.MockitoSugar
    +
    +import org.apache.spark._
    +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 blacklistTracker: BlacklistTracker = _
    +
    +  override def afterEach(): Unit = {
    +    if (blacklistTracker != null) {
    +      blacklistTracker = null
    +    }
    +    super.afterEach()
    +  }
    +
    +  val allOptions = (('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 = {
    +    allOptions.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
    +  }
    +
    +  test("Blacklisting individual tasks") {
    +    val conf = new SparkConf().setAppName("test").setMaster("local")
    +      .set(config.BLACKLIST_ENABLED.key, "true")
    +    val scheduler = mockTaskSchedWithConf(conf)
    +    // 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)
    +    tsm.updateBlacklistForFailedTask("hostA", "1", 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(blacklistTracker.nodeBlacklist() === Set())
    +    assertEquivalentToSet(blacklistTracker.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,
    +    // 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)
    --- End diff --
    
    good point, sorry about that.  In this case a search and replace is easy enough so I'll go ahead and update.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r77297304
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala ---
    @@ -0,0 +1,503 @@
    +/*
    + * 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.BeforeAndAfterEach
    +import org.scalatest.mock.MockitoSugar
    +
    +import org.apache.spark._
    +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 blacklistTracker: BlacklistTracker = _
    +
    +  override def afterEach(): Unit = {
    +    if (blacklistTracker != null) {
    +      blacklistTracker = null
    +    }
    +    super.afterEach()
    +  }
    +
    +  val allOptions = (('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 = {
    +    allOptions.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
    +  }
    +
    +  test("Blacklisting individual tasks") {
    +    val conf = new SparkConf().setAppName("test").setMaster("local")
    +      .set(config.BLACKLIST_ENABLED.key, "true")
    +    val scheduler = mockTaskSchedWithConf(conf)
    +    // 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)
    +    tsm.updateBlacklistForFailedTask("hostA", "1", 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(blacklistTracker.nodeBlacklist() === Set())
    +    assertEquivalentToSet(blacklistTracker.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,
    +    // 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)
    +    // 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 badPart = (index == 0 || index == 1)
    +        val taskExp = (badExec && badPart)
    +        assert(
    +          tsm.isExecutorBlacklistedForTask(executor, index) === taskExp)
    +        val executorExp = badExec
    +        assert(tsm.isExecutorBlacklistedForTaskSet(executor) === executorExp)
    +      }
    +    }
    +    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())
    +
    +    // 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()
    +  }
    +
    +  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)
    +    blacklistTracker = new BlacklistTracker(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 =>
    +      val taskSet = FakeTask.createTaskSet(1)
    +      val tsm = new TaskSetManager(scheduler, Some(tracker), taskSet, 4, clock)
    +      tsm.updateBlacklistForFailedTask("hostA", "1", 0)
    +      tracker.updateBlacklistForSuccessfulTaskSet(stage, 0, tsm.execToFailures)
    +    }
    +    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 =>
    +      val taskSet = FakeTask.createTaskSet(1)
    +      val tsm = new TaskSetManager(scheduler, Some(tracker), taskSet, 4, clock)
    +      tsm.updateBlacklistForFailedTask("hostA", "1", 0)
    +    }
    +    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)
    +      val taskSet = FakeTask.createTaskSet(4, stageId, 0)
    +      val tsm = new TaskSetManager(scheduler, Some(tracker), taskSet, 4, clock)
    +      (0 until 4).foreach { partition =>
    --- End diff --
    
    partition -> index


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r70726798
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -280,11 +304,25 @@ private[spark] class TaskSchedulerImpl(
           }
         }
         if (!launchedTask) {
    -      taskSet.abortIfCompletelyBlacklisted(executorIdToHost.keys)
    +      blacklistTracker.foreach(taskSet.abortIfCompletelyBlacklisted(executorsByHost, _))
         }
         return launchedTask
       }
     
    +  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
    --- End diff --
    
    Feels like you could use `executorsByHost.exists` here instead of this funny-looking return.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r71412204
  
    --- Diff: docs/configuration.md ---
    @@ -1149,6 +1149,20 @@ Apart from these, the following properties are also available, and may be useful
       </td>
     </tr>
     <tr>
    +  <td><code>spark.scheduler.blacklist.enabled</code></td>
    +  <td>true</td>
    --- End diff --
    
    I'm actually making it true by default, except for local mode.  
    
    https://github.com/squito/spark/blob/351a9a7e2893a0b90c57233d5e44a52c147bb2a8/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala#L195
    
    Too big a behavior change?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] Application Level Blacklisting

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/69092/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r70848996
  
    --- Diff: yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala ---
    @@ -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
    --- End diff --
    
    we probably want to remove this from our actual asks (locality preferences) as well.  the capacity scheduler (didn't check fair scheduler) is smart enough to not schedule on blacklisted node, but it would make sense to remove and potentially add another one that would be local rather then letting it fall back to pick a non-local one for us.  Not super critical so if we want to move that to another jira I'm fine with it. I think you had some other things around integrating it more with the resource managers later.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r72513868
  
    --- 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")
    +        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 (executorIdToBlacklistStatus.nonEmpty) {
    +        nextExpiryTime = executorIdToBlacklistStatus.map{_._2.expiryTime}.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(failuresByExec: HashMap[String, FailureStatus]): Unit = {
    --- End diff --
    
    maybe change this to "updateBlacklistForSuccessfulTaskSet"? More verbose but makes it more clear what this is doing


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r72541736
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala ---
    @@ -592,7 +599,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 = {
    --- End diff --
    
    looks like you don't need to pass this in anymore, since it's now part of the class?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #62621 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62621/consoleFull)** for PR 14079 at commit [`dbf904e`](https://github.com/apache/spark/commit/dbf904e80ad892b76d29bf0092db5810d14b3271).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r70846432
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala ---
    @@ -800,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) {
    --- End diff --
    
    not sure it matters to much here other then user information but if a node is already blacklisted, it could keep printing the info and adding it to the blacklistedNodes list.  I'm seeing this when one node fails that has a bunch of tasks running on it and all the tasks fail very quickly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/14079
  
    **[Test build #62245 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62245/consoleFull)** for PR 14079 at commit [`338db65`](https://github.com/apache/spark/commit/338db65e75c57bdc4af34ee342bc4ab843468b90).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r72537255
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala ---
    @@ -83,8 +85,15 @@ 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 execToFailures: HashMap[String, FailureStatus] = 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()
    --- End diff --
    
    nodeToBlacklistedTasks


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #14079: [SPARK-8425][CORE] New Blacklist Mechanism

Posted by kayousterhout <gi...@git.apache.org>.
Github user kayousterhout commented on a diff in the pull request:

    https://github.com/apache/spark/pull/14079#discussion_r72523093
  
    --- 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")
    +        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 (executorIdToBlacklistStatus.nonEmpty) {
    +        nextExpiryTime = executorIdToBlacklistStatus.map{_._2.expiryTime}.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(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
    +        executorIdToBlacklistStatus.put(exec, BlacklistedExecutor(node, expiryTime))
    +        val blacklistedExecsOnNode = nodeToFailedExecs.getOrElseUpdate(node, HashSet[String]())
    +        blacklistedExecsOnNode += exec
    +        executorIdToFailureCount.remove(exec)
    +        if (expiryTime < nextExpiryTime) {
    +          nextExpiryTime = expiryTime
    +        }
    +
    +        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)
    --- End diff --
    
    For readability maybe just move this to line 102? It's harmless (and negligible extra time) in the case when the executor needs to be blacklisted since then it will be removed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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