You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by dhruve <gi...@git.apache.org> on 2018/08/30 18:24:01 UTC

[GitHub] spark pull request #22288: [SPARK-22148] Acquire new executors to avoid hang...

GitHub user dhruve opened a pull request:

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

    [SPARK-22148] Acquire new executors to avoid hang because of blacklisting

    ## What changes were proposed in this pull request?
    Every time a task is unschedulable because of the condition where no. of task failures < no. of executors available, we currently abort the taskSet - failing the job. This change tries to acquire new executors if dynamicAllocation is turned on so that we can complete the job successfully.
    
    ## How was this patch tested?
    
    I performed some manual tests to check and validate the behavior. 
    
    ```scala
    val rdd = sc.parallelize(Seq(1 to 10), 3)
    
    import org.apache.spark.TaskContext
    
    val mapped = rdd.mapPartitionsWithIndex ( (index, iterator) => { if (index == 2) { Thread.sleep(30 * 1000); val attemptNum = TaskContext.get.attemptNumber; if (attemptNum < 3) throw new Exception("Fail for blacklisting")};  iterator.toList.map (x => x + " -> " + index).iterator } )
    
    mapped.collect
    ```
    
    Note: I am putting up this PR as initial draft to review the approach. 
    
    Todo List:
    - Add unit tests
    - Agree upon the conf name & value and update the docs 
    
    We can build on this approach further by:
    - Taking into account static allocation
    - Querying the RM to figure out if its a small cluster, then try to wait some more or abort immediately.
    - Try to distinguish between waiting for time while you acquire an executor and time for being unable to schedule a task.
    
    Open to suggestions.


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

    $ git pull https://github.com/dhruve/spark bug/SPARK-22148

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

    https://github.com/apache/spark/pull/22288.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 #22288
    
----
commit 5253b3134119b2a28cdaa1406d7bafb55f62cbc1
Author: Dhruve Ashar <dh...@...>
Date:   2018-08-30T18:08:58Z

    [SPARK-22148] Acquire new executors to avoid hang because of blacklisting

----


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #97176 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97176/testReport)** for PR 22288 at commit [`4ce7610`](https://github.com/apache/spark/commit/4ce7610522927ecabd0b8a75fc0a557938135b0d).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #96443 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/96443/testReport)** for PR 22288 at commit [`4c88168`](https://github.com/apache/spark/commit/4c881680fdde32244030b54b44125ac217dacb0d).


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #97943 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97943/testReport)** for PR 22288 at commit [`9b2aeaf`](https://github.com/apache/spark/commit/9b2aeaffdf6fc8d76b6c8ba2978c5dc7d6022899).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    > If it takes more time to acquire a new executor after killing a blacklisted one and the abort timer is up, we end up aborting the TaskSet. This was to see if we want to account for the time elapsed which doesn't include the time it took to obtain a new executor. Or we could just set the abortTimer expiration interval to a higher default value which should cover most of the cases.
    
    yeah I'm not sure you can do much better.  What if it takes forever to get a new executor?  There's no guarantee you will get anything else.  I don't see much value in adding another timer for that case, but happy to hear about an alternative.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r228636999
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +420,54 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case Some(taskIndex) => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we try to find an existing idle blacklisted
    +              // executor. If we cannot find one, we abort immediately. Else we kill the idle
    +              // executor and kick off an abortTimer which if it doesn't schedule a task within the
    +              // the timeout will abort the taskSet if we were unable to schedule any task from the
    +              // taskSet.
    +              // Note 1: We keep track of schedulability on a per taskSet basis rather than on a per
    +              // task basis.
    +              // Note 2: The taskSet can still be aborted when there are more than one idle
    +              // blacklisted executors and dynamic allocation is on. This can happen when a killed
    +              // idle executor isn't replaced in time by ExecutorAllocationManager as it relies on
    +              // pending tasks and doesn't kill executors on idle timeouts, resulting in the abort
    +              // timer to expire and abort the taskSet.
    +              executorIdToRunningTaskIds.find(x => !isExecutorBusy(x._1)) match {
    +                case Some ((executorId, _)) =>
    +                  if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) {
    +                    blacklistTrackerOpt.foreach(blt => blt.killBlacklistedIdleExecutor(executorId))
    +
    +                    val timeout = conf.get(config.UNSCHEDULABLE_TASKSET_TIMEOUT) * 1000
    +                    unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis() + timeout
    +                    logInfo(s"Waiting for $timeout ms for completely "
    +                      + s"blacklisted task to be schedulable again before aborting $taskSet.")
    +                    abortTimer.schedule(
    +                      createUnschedulableTaskSetAbortTimer(taskSet, taskIndex), timeout)
    +                  }
    +                case _ => // Abort Immediately
    --- End diff --
    
    Makes sense. Will update it.


---

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


[GitHub] spark issue #22288: [SPARK-22148][Scheduler] Acquire new executors to avoid ...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/3832/
    Test PASSed.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][Scheduler] Acquire new executors to avoid ...

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

    https://github.com/apache/spark/pull/22288
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/2706/
    Test PASSed.


---

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


[GitHub] spark issue #22288: [SPARK-22148][Scheduler] Acquire new executors to avoid ...

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

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


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r226999389
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -453,6 +503,22 @@ private[spark] class TaskSchedulerImpl(
         return tasks
       }
     
    +  private def getAbortTimer(taskSet: TaskSetManager, taskIndex: Int, timeout: Long): TimerTask = {
    --- End diff --
    
    perhaps rename to createUnschedulableTaskSetAbortTimer


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    you're still mentioning dynamic allocation in the description, but this isn't actually dynamic allocation specific, is it?


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #97469 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97469/testReport)** for PR 22288 at commit [`2c5a753`](https://github.com/apache/spark/commit/2c5a75354d36d08199b9805a7513a4ec4a546a27).


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r224105069
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -146,21 +146,31 @@ private[scheduler] class BlacklistTracker (
         nextExpiryTime = math.min(execMinExpiry, nodeMinExpiry)
       }
     
    +  private def killExecutor(exec: String, msg: String): Unit = {
    +    allocationClient match {
    +      case Some(a) =>
    +        logInfo(msg)
    +        a.killExecutors(Seq(exec), adjustTargetNumExecutors = false, countFailures = false,
    +          force = true)
    +      case None =>
    +        logWarning(s"Not attempting to kill blacklisted executor id $exec " +
    --- End diff --
    
    this should probably be info message since its not really anything bad they just aren't using dynamic allocation.


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r224113173
  
    --- Diff: docs/configuration.md ---
    @@ -1583,6 +1583,14 @@ Apart from these, the following properties are also available, and may be useful
         driver using more memory.
       </td>
     </tr>
    +<tr>
    +  <td><code>spark.scheduler.blacklist.unschedulableTaskSetTimeout</code></td>
    +  <td>120s</td>
    +  <td>
    +    The timeout in seconds to wait to try to acquire a new executor and schedule a task before
    --- End diff --
    
    remove "to try"
    
    I think change "which was previously" to "which is"



---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r228609353
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala ---
    @@ -503,6 +507,181 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
         verify(tsm).abort(anyString(), anyObject())
       }
     
    +  test("SPARK-22148 abort timer should kick in when task is completely blacklisted & no new " +
    +      "executor can be acquired") {
    +    // set the abort timer to fail immediately
    +    taskScheduler = setupSchedulerWithMockTaskSetBlacklist(
    +      config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "0")
    +
    +    // We have only 1 task remaining with 1 executor
    +    val taskSet = FakeTask.createTaskSet(numTasks = 1, stageAttemptId = 0)
    +    taskScheduler.submitTasks(taskSet)
    +    val tsm = stageToMockTaskSetManager(0)
    +
    +    // submit an offer with one executor
    +    val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten
    +
    +    // Fail the running task
    +    val failedTask = firstTaskAttempts.find(_.executorId == "executor0").get
    +    taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0))
    +    // we explicitly call the handleFailedTask method here to avoid adding a sleep in the test suite
    +    // Reason being - handleFailedTask is run by an executor service and there is a momentary delay
    +    // before it is launched and this fails the assertion check.
    +    tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, UnknownReason)
    +    when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask(
    +      "executor0", failedTask.index)).thenReturn(true)
    +
    +    // make an offer on the blacklisted executor.  We won't schedule anything, and set the abort
    +    // timer to kick in immediately
    +    assert(taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten.size === 0)
    +    // Wait for the abort timer to kick in. Without sleep the test exits before the timer is
    +    // triggered.
    +    eventually(timeout(500.milliseconds)) {
    +      assert(tsm.isZombie)
    +    }
    +  }
    +
    +  test("SPARK-22148 try to acquire a new executor when task is unschedulable with 1 executor") {
    +    taskScheduler = setupSchedulerWithMockTaskSetBlacklist(
    +      config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "10")
    +
    +    // We have only 1 task remaining with 1 executor
    +    val taskSet = FakeTask.createTaskSet(numTasks = 1, stageAttemptId = 0)
    +    taskScheduler.submitTasks(taskSet)
    +    val tsm = stageToMockTaskSetManager(0)
    +
    +    // submit an offer with one executor
    +    val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten
    +
    +    // Fail the running task
    +    val failedTask = firstTaskAttempts.find(_.executorId == "executor0").get
    +    taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0))
    +    // we explicitly call the handleFailedTask method here to avoid adding a sleep in the test suite
    +    // Reason being - handleFailedTask is run by an executor service and there is a momentary delay
    +    // before it is launched and this fails the assertion check.
    +    tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, UnknownReason)
    +    when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask(
    +      "executor0", failedTask.index)).thenReturn(true)
    +
    +    // make an offer on the blacklisted executor.  We won't schedule anything, and set the abort
    +    // timer to expire if no new executors could be acquired. We kill the existing idle blacklisted
    +    // executor and try to acquire a new one.
    +    assert(taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten.size === 0)
    +    assert(!tsm.isZombie)
    +
    +    // Offer a new executor which should be accepted
    +    assert(taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor1", "host0", 1)
    +    )).flatten.size === 1)
    +
    +    assert(!tsm.isZombie)
    +  }
    +
    +  // This is to test a scenario where we have two taskSets completely blacklisted and on acquiring
    +  // a new executor we don't want the abort timer for the second taskSet to expire and abort the job
    +  test("SPARK-22148 abort timer should clear unschedulableTaskSetToExpiryTime for all TaskSets") {
    +    taskScheduler = setupSchedulerWithMockTaskSetBlacklist(
    +      config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "10")
    +
    +    // We have 2 taskSets with 1 task remaining in each with 1 executor completely blacklisted
    +    val taskSet1 = FakeTask.createTaskSet(numTasks = 1, stageId = 0, stageAttemptId = 0)
    +    taskScheduler.submitTasks(taskSet1)
    +    val taskSet2 = FakeTask.createTaskSet(numTasks = 1, stageId = 1, stageAttemptId = 0)
    +    taskScheduler.submitTasks(taskSet2)
    +    val tsm = stageToMockTaskSetManager(0)
    +
    +    // submit an offer with one executor
    +    val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten
    +
    +    assert(taskScheduler.unschedulableTaskSetToExpiryTime.size == 0)
    +
    +    // Fail the running task
    +    val failedTask = firstTaskAttempts.find(_.executorId == "executor0").get
    +    taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0))
    +    tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, UnknownReason)
    +    when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask(
    +      "executor0", failedTask.index)).thenReturn(true)
    +
    +    // make an offer. We will schedule the task from the second taskSet. Since a task was scheduled
    +    // we do not kick off the abort timer for taskSet1
    +    val secondTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten
    +
    +    assert(taskScheduler.unschedulableTaskSetToExpiryTime.size == 0)
    +
    +    val tsm2 = stageToMockTaskSetManager(1)
    +    val failedTask2 = secondTaskAttempts.find(_.executorId == "executor0").get
    +    taskScheduler.statusUpdate(failedTask2.taskId, TaskState.FAILED, ByteBuffer.allocate(0))
    +    tsm2.handleFailedTask(failedTask2.taskId, TaskState.FAILED, UnknownReason)
    +    when(tsm2.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask(
    +      "executor0", failedTask2.index)).thenReturn(true)
    +
    +    // make an offer on the blacklisted executor.  We won't schedule anything, and set the abort
    +    // timer for taskSet1 and taskSet2
    +    assert(taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten.size === 0)
    +
    +    assert(taskScheduler.unschedulableTaskSetToExpiryTime.size == 2)
    +
    +    // Offer a new executor which should be accepted
    +    assert(taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor1", "host1", 1)
    +    )).flatten.size === 1)
    +
    +    // Check if all the taskSets are cleared
    +    assert(taskScheduler.unschedulableTaskSetToExpiryTime.size == 0)
    +
    +    assert(!tsm.isZombie)
    +  }
    +
    +  // this test is to check that we don't abort a taskSet which is not being scheduled on other
    +  // executors as it is waiting on locality timeout and not being aborted because it is still not
    +  // completely blacklisted.
    +  test("SPARK-22148 Ensure we don't abort the taskSet if we haven't been completely blacklisted") {
    +    taskScheduler = setupSchedulerWithMockTaskSetBlacklist(
    +      config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "0")
    +
    +    val preferredLocation = Seq(ExecutorCacheTaskLocation("host0", "executor0"))
    +    val taskSet1 = FakeTask.createTaskSet(numTasks = 1, stageId = 0, stageAttemptId = 0,
    +      preferredLocation)
    +    taskScheduler.submitTasks(taskSet1)
    +
    +    val tsm = stageToMockTaskSetManager(0)
    +
    +    // submit an offer with one executor
    +    var taskAttempts = taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten
    +
    +    // Fail the running task
    +    val failedTask = taskAttempts.find(_.executorId == "executor0").get
    --- End diff --
    
    same here


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r225198423
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala ---
    @@ -503,6 +505,89 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
         verify(tsm).abort(anyString(), anyObject())
       }
     
    +  test("SPARK-22148 abort timer should kick in when task is completely blacklisted & no new " +
    +    "executor can be acquired") {
    +    // set the abort timer to fail immediately
    +    taskScheduler = setupSchedulerWithMockTaskSetBlacklist(
    +      config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "0")
    +
    +    // We have only 1 task remaining with 1 executor
    +    val taskSet = FakeTask.createTaskSet(numTasks = 1, stageAttemptId = 0)
    +    taskScheduler.submitTasks(taskSet)
    +    val tsm = stageToMockTaskSetManager(0)
    +
    +    // submit an offer with one executor
    +    val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten
    +
    +    // Fail the running task
    +    val failedTask = firstTaskAttempts.find(_.executorId == "executor0").get
    +    taskScheduler.statusUpdate(
    +      tid = failedTask.taskId,
    +      state = TaskState.FAILED,
    +      serializedData = ByteBuffer.allocate(0)
    +    )
    +    // Wait for the failed task to propagate.
    +    Thread.sleep(500)
    +
    +    when(stageToMockTaskSetBlacklist(0).isExecutorBlacklistedForTask("executor0", failedTask.index))
    --- End diff --
    
    Can we perhaps mock the blacklisted tracker to mark it as blacklisted to get rid of the sleep here?  


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r224111068
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -117,6 +117,10 @@ private[spark] class TaskSchedulerImpl(
     
       protected val executorIdToHost = new HashMap[String, String]
     
    +  private val abortTimer = new Timer(true)
    --- End diff --
    
    we should cancel the timer in the stop() call


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r228609147
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala ---
    @@ -503,6 +507,181 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
         verify(tsm).abort(anyString(), anyObject())
       }
     
    +  test("SPARK-22148 abort timer should kick in when task is completely blacklisted & no new " +
    +      "executor can be acquired") {
    +    // set the abort timer to fail immediately
    +    taskScheduler = setupSchedulerWithMockTaskSetBlacklist(
    +      config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "0")
    +
    +    // We have only 1 task remaining with 1 executor
    +    val taskSet = FakeTask.createTaskSet(numTasks = 1, stageAttemptId = 0)
    +    taskScheduler.submitTasks(taskSet)
    +    val tsm = stageToMockTaskSetManager(0)
    +
    +    // submit an offer with one executor
    +    val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten
    +
    +    // Fail the running task
    +    val failedTask = firstTaskAttempts.find(_.executorId == "executor0").get
    +    taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0))
    +    // we explicitly call the handleFailedTask method here to avoid adding a sleep in the test suite
    +    // Reason being - handleFailedTask is run by an executor service and there is a momentary delay
    +    // before it is launched and this fails the assertion check.
    +    tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, UnknownReason)
    +    when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask(
    +      "executor0", failedTask.index)).thenReturn(true)
    +
    +    // make an offer on the blacklisted executor.  We won't schedule anything, and set the abort
    +    // timer to kick in immediately
    +    assert(taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten.size === 0)
    +    // Wait for the abort timer to kick in. Without sleep the test exits before the timer is
    +    // triggered.
    +    eventually(timeout(500.milliseconds)) {
    +      assert(tsm.isZombie)
    +    }
    +  }
    +
    +  test("SPARK-22148 try to acquire a new executor when task is unschedulable with 1 executor") {
    +    taskScheduler = setupSchedulerWithMockTaskSetBlacklist(
    +      config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "10")
    +
    +    // We have only 1 task remaining with 1 executor
    +    val taskSet = FakeTask.createTaskSet(numTasks = 1, stageAttemptId = 0)
    +    taskScheduler.submitTasks(taskSet)
    +    val tsm = stageToMockTaskSetManager(0)
    +
    +    // submit an offer with one executor
    +    val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten
    +
    +    // Fail the running task
    +    val failedTask = firstTaskAttempts.find(_.executorId == "executor0").get
    +    taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0))
    +    // we explicitly call the handleFailedTask method here to avoid adding a sleep in the test suite
    +    // Reason being - handleFailedTask is run by an executor service and there is a momentary delay
    +    // before it is launched and this fails the assertion check.
    +    tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, UnknownReason)
    +    when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask(
    +      "executor0", failedTask.index)).thenReturn(true)
    +
    +    // make an offer on the blacklisted executor.  We won't schedule anything, and set the abort
    +    // timer to expire if no new executors could be acquired. We kill the existing idle blacklisted
    +    // executor and try to acquire a new one.
    +    assert(taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten.size === 0)
    +    assert(!tsm.isZombie)
    +
    +    // Offer a new executor which should be accepted
    +    assert(taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor1", "host0", 1)
    +    )).flatten.size === 1)
    +
    +    assert(!tsm.isZombie)
    +  }
    +
    +  // This is to test a scenario where we have two taskSets completely blacklisted and on acquiring
    +  // a new executor we don't want the abort timer for the second taskSet to expire and abort the job
    +  test("SPARK-22148 abort timer should clear unschedulableTaskSetToExpiryTime for all TaskSets") {
    +    taskScheduler = setupSchedulerWithMockTaskSetBlacklist(
    +      config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "10")
    +
    +    // We have 2 taskSets with 1 task remaining in each with 1 executor completely blacklisted
    +    val taskSet1 = FakeTask.createTaskSet(numTasks = 1, stageId = 0, stageAttemptId = 0)
    +    taskScheduler.submitTasks(taskSet1)
    +    val taskSet2 = FakeTask.createTaskSet(numTasks = 1, stageId = 1, stageAttemptId = 0)
    +    taskScheduler.submitTasks(taskSet2)
    +    val tsm = stageToMockTaskSetManager(0)
    +
    +    // submit an offer with one executor
    +    val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten
    +
    +    assert(taskScheduler.unschedulableTaskSetToExpiryTime.size == 0)
    +
    +    // Fail the running task
    +    val failedTask = firstTaskAttempts.find(_.executorId == "executor0").get
    +    taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0))
    +    tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, UnknownReason)
    +    when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask(
    +      "executor0", failedTask.index)).thenReturn(true)
    +
    +    // make an offer. We will schedule the task from the second taskSet. Since a task was scheduled
    +    // we do not kick off the abort timer for taskSet1
    +    val secondTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten
    +
    +    assert(taskScheduler.unschedulableTaskSetToExpiryTime.size == 0)
    +
    +    val tsm2 = stageToMockTaskSetManager(1)
    +    val failedTask2 = secondTaskAttempts.find(_.executorId == "executor0").get
    --- End diff --
    
    minor, you've only got one taskattempt here, you could just do
    
    ```scala
    val failedTask2 = secondTaskAttempts.head
    ```


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    @squito for the locality wait, it would be the same as the condition where it is not completely blacklisted. I have added a test for this. If we want to ensure the sequence for the timeout expiring and the task being scheduled, we will have to add some more delay. Let me know if we want to do it, or the test seems to suffice.


---

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


[GitHub] spark pull request #22288: [SPARK-22148][Scheduler] Acquire new executors to...

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

    https://github.com/apache/spark/pull/22288#discussion_r216726323
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -414,9 +425,48 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    -        }
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case taskIndex: Some[Int] => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we kill an existing blacklisted executor/s and
    +              // kick off an abortTimer which after waiting will abort the taskSet if we were
    +              // unable to schedule any task from the taskSet.
    +              // Note: We keep a track of schedulability on a per taskSet basis rather than on a
    +              // per task basis.
    +              val executor = hostToExecutors.valuesIterator.next().iterator.next()
    +              logDebug("Killing executor because of task unschedulability: " + executor)
    +              blacklistTrackerOpt.foreach(blt => blt.killBlacklistedExecutor(executor))
    +
    +              if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) {
    +                  unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis()
    +                  abortTimer.schedule(new TimerTask() {
    +                    override def run() {
    +                      if (unschedulableTaskSetToExpiryTime.contains(taskSet) &&
    +                        (unschedulableTaskSetToExpiryTime(taskSet)
    +                          + UNSCHEDULABLE_TASKSET_TIMEOUT_MS)
    +                          <= clock.getTimeMillis()
    +                      ) {
    +                        logInfo("Cannot schedule any task because of complete blacklisting. " +
    +                          "Wait time for scheduling expired. Aborting the application.")
    +                        taskSet.abortSinceCompletelyBlacklisted(taskIndex.get)
    +                      } else {
    +                        this.cancel()
    +                      }
    +                    }
    +                  }, UNSCHEDULABLE_TASKSET_TIMEOUT_MS)
    +                }
    +            case _ => // Do nothing.
    +            }
    +          } else {
    +            // If a task was scheduled, we clear the expiry time for the taskSet. The abort timer
    +            // checks this entry to decide if we want to abort the taskSet.
    +            if (unschedulableTaskSetToExpiryTime.contains(taskSet)) {
    --- End diff --
    
    you can move this up to the `else` so its an `else if`.  Or you could also just call `remove` without checking `contains`, that avoids probing twice.


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r226754849
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +420,65 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case taskIndex: Some[Int] => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we try to find an existing idle blacklisted
    +              // executor. If we cannot find one, we abort immediately. Else we kill the idle
    +              // executor and kick off an abortTimer which if it doesn't schedule a task within the
    +              // the timeout will abort the taskSet if we were unable to schedule any task from the
    +              // taskSet.
    +              // Note 1: We keep track of schedulability on a per taskSet basis rather than on a per
    +              // task basis.
    +              // Note 2: The taskSet can still be aborted when there are more than one idle
    +              // blacklisted executors and dynamic allocation is on. This can happen when a killed
    +              // idle executor isn't replaced in time by ExecutorAllocationManager as it relies on
    +              // pending tasks and doesn't kill executors on idle timeouts, resulting in the abort
    +              // timer to expire and abort the taskSet.
    +              executorIdToRunningTaskIds.find(x => !isExecutorBusy(x._1)) match {
    +                case Some (x) =>
    +                  val executorId = x._1
    +                  if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) {
    +                    blacklistTrackerOpt.foreach(blt => blt.killBlacklistedIdleExecutor(executorId))
    +
    +                    unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis()
    +                    val timeout = conf.get(config.UNSCHEDULABLE_TASKSET_TIMEOUT) * 1000
    +                    logInfo(s"Waiting for $timeout ms for completely "
    +                      + s"blacklisted task to be schedulable again before aborting $taskSet.")
    +                    abortTimer.schedule(new TimerTask() {
    +                      override def run() {
    +                        if (unschedulableTaskSetToExpiryTime.contains(taskSet) &&
    +                          (unschedulableTaskSetToExpiryTime(taskSet) + timeout)
    +                            <= clock.getTimeMillis()
    +                        ) {
    +                          logInfo("Cannot schedule any task because of complete blacklisting. " +
    +                            s"Wait time for scheduling expired. Aborting $taskSet.")
    +                          taskSet.abortSinceCompletelyBlacklisted(taskIndex.get)
    +                        } else {
    +                          this.cancel()
    +                        }
    +                      }
    +                    }, timeout)
    +                  }
    +                case _ => // Abort Immediately
    +                  logInfo("Cannot schedule any task because of complete blacklisting. No idle" +
    +                    s" executors can be found to kill. Aborting $taskSet." )
    +                  taskSet.abortSinceCompletelyBlacklisted(taskIndex.get)
    +              }
    +            case _ => // Do nothing if no tasks completely blacklisted.
    +          }
    +        } else {
    +          // If a task was scheduled, we clear the expiry time for all the taskSets. This ensures
    +          // that we have got atleast a non blacklisted executor and the job can progress. The
    +          // abort timer checks this entry to decide if we want to abort the taskSet.
    --- End diff --
    
    That is correct. It also covers other scenario that @tgravescs originally pointed out. 
    
    Lets say if you have multiple taskSets running which are completely blacklisted. If you were able to get an executor, you would just clear the timer for that specific taskSet. Now due to resource constraint, if you weren't able to obtain another executor within the timeout for the other taskSet, you would abort the other taskSet when you could actually wait for it to be scheduled on the newly obtained executor.
    
    So clearing the timer for all the taskSets ensures that currently we aren't in a completely blacklisted state and should try to run to completion. However if the taskset itself is flawed, we would eventually fail. This could result in wasted effort, but we don't have a way to determine that yet, so this should be okay.  


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #97556 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97556/testReport)** for PR 22288 at commit [`d2af73d`](https://github.com/apache/spark/commit/d2af73d4bd10676169f4fddc9dd262c97c0a9967).


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    test this please


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #97208 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97208/testReport)** for PR 22288 at commit [`c361693`](https://github.com/apache/spark/commit/c361693a3d08a1bea1d2919f0a8e970c03959cc8).


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r222785870
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -117,6 +117,12 @@ private[spark] class TaskSchedulerImpl(
     
       protected val executorIdToHost = new HashMap[String, String]
     
    +  private val abortTimer = new Timer(true)
    +
    --- End diff --
    
    remove unneeded newline


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    As I mentioned at https://github.com/apache/spark/pull/22288#discussion_r216874530, I'm quite worry about this killing behaviour.  I thik we should kill a executor iff it is idle.
    
    By looking through dissuction above, give my thoughts below:
    
    * with dynamic allocation
    
    Maybe, we can add `onTaskCompletelyBlacklisted()` method in DA manager's `Listener` and pass a e.g. `TaskCompletelyBlacklistedEvent` to it. Thus, DA manger will allocate new executor for us.
    
    * with static allocation
    
    Set `spark.scheduler.unschedulableTaskSetTimeout` for a `TaskSet`. If a task blacklisted completely, 
    kill some executors iff they're idle (Maybe, taking executors' allocation time into acount here, we should increase timeout upperbound for a little for this `TaskSet`.).  Then, waiting until to be scheduled or timeout&abort.  


---

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


[GitHub] spark issue #22288: [SPARK-22148][Scheduler] Acquire new executors to avoid ...

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

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


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r222814235
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +421,63 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    -        }
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case taskIndex: Some[Int] => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we try to find an existing idle blacklisted
    +              // executor. If we cannot find one, we abort immediately. Else we kill the idle
    +              // executor and kick off an abortTimer which after waiting will abort the taskSet if
    +              // we were unable to schedule any task from the taskSet.
    +              // Note 1: We keep a track of schedulability on a per taskSet basis rather than on a
    +              // per task basis.
    +              // Note 2: The taskSet can still be aborted when there are more than one idle
    +              // blacklisted executors and dynamic allocation is on. This is because we rely on the
    +              // ExecutorAllocationManager to acquire a new executor based on the pending tasks and
    +              // it won't release any blacklisted executors which idle timeout after we kill an
    +              // executor to acquire a new one, resulting in the abort timer to expire and abort the
    +              // taskSet.
    +              executorIdToRunningTaskIds.find(x => !isExecutorBusy(x._1)) match {
    +                case Some (x) =>
    +                  val executorId = x._1
    +                  if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) {
    +                    blacklistTrackerOpt.foreach(blt => blt.killBlacklistedIdleExecutor(executorId))
    +
    +                    unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis()
    +                    val timeout = conf.get(config.UNSCHEDULABLE_TASKSET_TIMEOUT) * 1000
    +                    logInfo(s"Waiting for $timeout ms for completely "
    +                      + s"blacklisted task to be schedulable again before aborting $taskSet.")
    +                    abortTimer.schedule(new TimerTask() {
    +                      override def run() {
    +                        if (unschedulableTaskSetToExpiryTime.contains(taskSet) &&
    +                          (unschedulableTaskSetToExpiryTime(taskSet) + timeout)
    +                            <= clock.getTimeMillis()
    +                        ) {
    +                          logInfo("Cannot schedule any task because of complete blacklisting. " +
    +                            s"Wait time for scheduling expired. Aborting $taskSet.")
    +                          taskSet.abortSinceCompletelyBlacklisted(taskIndex.get)
    +                        } else {
    +                          this.cancel()
    +                        }
    +                      }
    +                    }, timeout)
    +                  }
    +                case _ => // Abort Immediately
    +                  logInfo("Cannot schedule any task because of complete blacklisting. No idle" +
    +                  s" executors could be found. Aborting $taskSet." )
    --- End diff --
    
    can be found to kill


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r227095389
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +420,55 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case Some(taskIndex) => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we try to find an existing idle blacklisted
    +              // executor. If we cannot find one, we abort immediately. Else we kill the idle
    --- End diff --
    
    By clearing the abort timer as soon as a task is launched we are relaxing this situation. 
    If there are large backlog of tasks:
    - If we acquire new executors or launch new tasks we will defer the check
    - If we cannot acquire new executors and we are running with long running tasks such that no new tasks can be launched and we have less no. of executors compared to max failures, in that case this will end up being harsh. This can happen, but seems more like a very specific edge case.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    Failure is unrelated.


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r226478269
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala ---
    @@ -679,20 +679,28 @@ private[spark] class TaskSetManager(
                   }
                 }
               }
    +
               if (blacklistedEverywhere) {
    -            val partition = tasks(indexInTaskSet).partitionId
    -            abort(s"""
    -              |Aborting $taskSet because task $indexInTaskSet (partition $partition)
    -              |cannot run anywhere due to node and executor blacklist.
    -              |Most recent failure:
    -              |${taskSetBlacklist.getLatestFailureReason}
    -              |
    -              |Blacklisting behavior can be configured via spark.blacklist.*.
    -              |""".stripMargin)
    +            return Some(indexInTaskSet)
    --- End diff --
    
    not a big deal, but I think you can get rid of the early return and make this more idiomatic by changing it to
    
    ```scala
    taskSetBlacklistHelperOpt.flatMap { taskSetBlacklist =>
    ...
      pendingTask.find { indexInTaskSet =>
    ```


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r226476094
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +420,65 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case taskIndex: Some[Int] => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we try to find an existing idle blacklisted
    +              // executor. If we cannot find one, we abort immediately. Else we kill the idle
    +              // executor and kick off an abortTimer which if it doesn't schedule a task within the
    +              // the timeout will abort the taskSet if we were unable to schedule any task from the
    +              // taskSet.
    +              // Note 1: We keep track of schedulability on a per taskSet basis rather than on a per
    +              // task basis.
    +              // Note 2: The taskSet can still be aborted when there are more than one idle
    +              // blacklisted executors and dynamic allocation is on. This can happen when a killed
    +              // idle executor isn't replaced in time by ExecutorAllocationManager as it relies on
    +              // pending tasks and doesn't kill executors on idle timeouts, resulting in the abort
    +              // timer to expire and abort the taskSet.
    +              executorIdToRunningTaskIds.find(x => !isExecutorBusy(x._1)) match {
    +                case Some (x) =>
    +                  val executorId = x._1
    +                  if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) {
    +                    blacklistTrackerOpt.foreach(blt => blt.killBlacklistedIdleExecutor(executorId))
    +
    +                    unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis()
    +                    val timeout = conf.get(config.UNSCHEDULABLE_TASKSET_TIMEOUT) * 1000
    +                    logInfo(s"Waiting for $timeout ms for completely "
    +                      + s"blacklisted task to be schedulable again before aborting $taskSet.")
    +                    abortTimer.schedule(new TimerTask() {
    +                      override def run() {
    +                        if (unschedulableTaskSetToExpiryTime.contains(taskSet) &&
    +                          (unschedulableTaskSetToExpiryTime(taskSet) + timeout)
    +                            <= clock.getTimeMillis()
    +                        ) {
    +                          logInfo("Cannot schedule any task because of complete blacklisting. " +
    +                            s"Wait time for scheduling expired. Aborting $taskSet.")
    +                          taskSet.abortSinceCompletelyBlacklisted(taskIndex.get)
    +                        } else {
    +                          this.cancel()
    +                        }
    +                      }
    +                    }, timeout)
    +                  }
    +                case _ => // Abort Immediately
    +                  logInfo("Cannot schedule any task because of complete blacklisting. No idle" +
    +                    s" executors can be found to kill. Aborting $taskSet." )
    +                  taskSet.abortSinceCompletelyBlacklisted(taskIndex.get)
    +              }
    +            case _ => // Do nothing if no tasks completely blacklisted.
    +          }
    +        } else {
    +          // If a task was scheduled, we clear the expiry time for all the taskSets. This ensures
    +          // that we have got atleast a non blacklisted executor and the job can progress. The
    +          // abort timer checks this entry to decide if we want to abort the taskSet.
    +          if (unschedulableTaskSetToExpiryTime.nonEmpty) {
    +            logInfo("Clearing the expiry times for all unschedulable taskSets as")
    --- End diff --
    
    msg is incomplete


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/4549/
    Test PASSed.


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r224873268
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -146,21 +146,31 @@ private[scheduler] class BlacklistTracker (
         nextExpiryTime = math.min(execMinExpiry, nodeMinExpiry)
       }
     
    +  private def killExecutor(exec: String, msg: String): Unit = {
    +    allocationClient match {
    +      case Some(a) =>
    +        logInfo(msg)
    +        a.killExecutors(Seq(exec), adjustTargetNumExecutors = false, countFailures = false,
    +          force = true)
    +      case None =>
    +        logInfo(s"Not attempting to kill blacklisted executor id $exec " +
    +          s"since allocation client is not defined.")
    +    }
    +  }
    +
       private def killBlacklistedExecutor(exec: String): Unit = {
         if (conf.get(config.BLACKLIST_KILL_ENABLED)) {
    -      allocationClient match {
    -        case Some(a) =>
    -          logInfo(s"Killing blacklisted executor id $exec " +
    -            s"since ${config.BLACKLIST_KILL_ENABLED.key} is set.")
    -          a.killExecutors(Seq(exec), adjustTargetNumExecutors = false, countFailures = false,
    -            force = true)
    -        case None =>
    -          logWarning(s"Not attempting to kill blacklisted executor id $exec " +
    -            s"since allocation client is not defined.")
    -      }
    +      killExecutor(exec,
    +        s"Killing blacklisted executor id $exec since ${config.BLACKLIST_KILL_ENABLED.key} is set.")
         }
       }
     
    +  private[scheduler] def killBlacklistedIdleExecutor(exec: String): Unit = {
    +    killExecutor(exec,
    --- End diff --
    
    We want to kill an idle executor which is completely blacklisted without having to enable killing for all the blacklisted executors, so we made the change otherwise we would have kept it as is.


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r228608360
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala ---
    @@ -503,6 +507,145 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
         verify(tsm).abort(anyString(), anyObject())
       }
     
    +  test("SPARK-22148 abort timer should kick in when task is completely blacklisted & no new " +
    +    "executor can be acquired") {
    +    // set the abort timer to fail immediately
    +    taskScheduler = setupSchedulerWithMockTaskSetBlacklist(
    +      config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "0")
    +
    +    // We have only 1 task remaining with 1 executor
    +    val taskSet = FakeTask.createTaskSet(numTasks = 1, stageAttemptId = 0)
    +    taskScheduler.submitTasks(taskSet)
    +    val tsm = stageToMockTaskSetManager(0)
    +
    +    // submit an offer with one executor
    +    val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten
    +
    +    // Fail the running task
    +    val failedTask = firstTaskAttempts.find(_.executorId == "executor0").get
    +    taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0))
    +    // we explicitly call the handleFailedTask method here to avoid adding a sleep in the test suite
    +    // Reason being - handleFailedTask is run by an executor service and there is a momentary delay
    +    // before it is launched and this fails the assertion check.
    +    tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, UnknownReason)
    +    when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask(
    +      "executor0", failedTask.index)).thenReturn(true)
    +
    +    // make an offer on the blacklisted executor.  We won't schedule anything, and set the abort
    +    // timer to kick in immediately
    +    assert(taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten.size === 0)
    +    // Wait for the abort timer to kick in. Without sleep the test exits before the timer is
    +    // triggered.
    +    eventually(timeout(500.milliseconds)) {
    +      assert(tsm.isZombie)
    +    }
    +  }
    +
    +  test("SPARK-22148 try to acquire a new executor when task is unschedulable with 1 executor") {
    +    taskScheduler = setupSchedulerWithMockTaskSetBlacklist(
    +      config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "10")
    +
    +    // We have only 1 task remaining with 1 executor
    +    val taskSet = FakeTask.createTaskSet(numTasks = 1, stageAttemptId = 0)
    +    taskScheduler.submitTasks(taskSet)
    +    val tsm = stageToMockTaskSetManager(0)
    +
    +    // submit an offer with one executor
    +    val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten
    +
    +    // Fail the running task
    +    val failedTask = firstTaskAttempts.find(_.executorId == "executor0").get
    +    taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0))
    +    // we explicitly call the handleFailedTask method here to avoid adding a sleep in the test suite
    +    // Reason being - handleFailedTask is run by an executor service and there is a momentary delay
    +    // before it is launched and this fails the assertion check.
    +    tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, UnknownReason)
    +    when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask(
    +      "executor0", failedTask.index)).thenReturn(true)
    +
    +    // make an offer on the blacklisted executor.  We won't schedule anything, and set the abort
    +    // timer to expire if no new executors could be acquired. We kill the existing idle blacklisted
    +    // executor and try to acquire a new one.
    +    assert(taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten.size === 0)
    +    assert(!tsm.isZombie)
    +
    +    // Offer a new executor which should be accepted
    +    assert(taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor1", "host0", 1)
    +    )).flatten.size === 1)
    --- End diff --
    
    ok yeah in that case, might as well leave it as the default, does this help at all?
    
    Also I'd add asserts on `unschedulableTaskSetToExpiryTime` here -- `assert(unschedulableTaskSetToExpiryTime.contains(tsm))` after the first one and `assert(unschedulableTaskSetToExpiryTime.isEmpty)` after the second one, thats the really important part, as we're not expecting to cross the timeout and make it a zombie anyway.


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r227077071
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -453,6 +504,25 @@ private[spark] class TaskSchedulerImpl(
         return tasks
       }
     
    +  private def createUnschedulableTaskSetAbortTimer(
    +      taskSet: TaskSetManager,
    +      taskIndex: Int,
    +      timeout: Long): TimerTask = {
    --- End diff --
    
    good catch.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #97165 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97165/testReport)** for PR 22288 at commit [`43e0af2`](https://github.com/apache/spark/commit/43e0af2238855dc69c93b36e40493c3eda670e90).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r227080534
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala ---
    @@ -503,6 +507,145 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
         verify(tsm).abort(anyString(), anyObject())
       }
     
    +  test("SPARK-22148 abort timer should kick in when task is completely blacklisted & no new " +
    +    "executor can be acquired") {
    +    // set the abort timer to fail immediately
    +    taskScheduler = setupSchedulerWithMockTaskSetBlacklist(
    +      config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "0")
    +
    +    // We have only 1 task remaining with 1 executor
    +    val taskSet = FakeTask.createTaskSet(numTasks = 1, stageAttemptId = 0)
    +    taskScheduler.submitTasks(taskSet)
    +    val tsm = stageToMockTaskSetManager(0)
    +
    +    // submit an offer with one executor
    +    val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten
    +
    +    // Fail the running task
    +    val failedTask = firstTaskAttempts.find(_.executorId == "executor0").get
    +    taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0))
    +    // we explicitly call the handleFailedTask method here to avoid adding a sleep in the test suite
    +    // Reason being - handleFailedTask is run by an executor service and there is a momentary delay
    +    // before it is launched and this fails the assertion check.
    +    tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, UnknownReason)
    +    when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask(
    +      "executor0", failedTask.index)).thenReturn(true)
    +
    +    // make an offer on the blacklisted executor.  We won't schedule anything, and set the abort
    +    // timer to kick in immediately
    +    assert(taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten.size === 0)
    +    // Wait for the abort timer to kick in. Without sleep the test exits before the timer is
    +    // triggered.
    +    eventually(timeout(500.milliseconds)) {
    +      assert(tsm.isZombie)
    +    }
    +  }
    +
    +  test("SPARK-22148 try to acquire a new executor when task is unschedulable with 1 executor") {
    +    taskScheduler = setupSchedulerWithMockTaskSetBlacklist(
    +      config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "10")
    +
    +    // We have only 1 task remaining with 1 executor
    +    val taskSet = FakeTask.createTaskSet(numTasks = 1, stageAttemptId = 0)
    +    taskScheduler.submitTasks(taskSet)
    +    val tsm = stageToMockTaskSetManager(0)
    +
    +    // submit an offer with one executor
    +    val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten
    +
    +    // Fail the running task
    +    val failedTask = firstTaskAttempts.find(_.executorId == "executor0").get
    +    taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0))
    +    // we explicitly call the handleFailedTask method here to avoid adding a sleep in the test suite
    +    // Reason being - handleFailedTask is run by an executor service and there is a momentary delay
    +    // before it is launched and this fails the assertion check.
    +    tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, UnknownReason)
    +    when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask(
    +      "executor0", failedTask.index)).thenReturn(true)
    +
    +    // make an offer on the blacklisted executor.  We won't schedule anything, and set the abort
    +    // timer to expire if no new executors could be acquired. We kill the existing idle blacklisted
    +    // executor and try to acquire a new one.
    +    assert(taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten.size === 0)
    +    assert(!tsm.isZombie)
    +
    +    // Offer a new executor which should be accepted
    +    assert(taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor1", "host0", 1)
    +    )).flatten.size === 1)
    --- End diff --
    
    We specify the config in `seconds`. The expectation here is that timer should expire in `10 seconds`, which I think is sufficient to account for gc time. However, we could just remove this as the default is 120s. 


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r228678209
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +420,54 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case Some(taskIndex) => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we try to find an existing idle blacklisted
    +              // executor. If we cannot find one, we abort immediately. Else we kill the idle
    +              // executor and kick off an abortTimer which if it doesn't schedule a task within the
    +              // the timeout will abort the taskSet if we were unable to schedule any task from the
    +              // taskSet.
    +              // Note 1: We keep track of schedulability on a per taskSet basis rather than on a per
    +              // task basis.
    +              // Note 2: The taskSet can still be aborted when there are more than one idle
    +              // blacklisted executors and dynamic allocation is on. This can happen when a killed
    +              // idle executor isn't replaced in time by ExecutorAllocationManager as it relies on
    +              // pending tasks and doesn't kill executors on idle timeouts, resulting in the abort
    +              // timer to expire and abort the taskSet.
    +              executorIdToRunningTaskIds.find(x => !isExecutorBusy(x._1)) match {
    +                case Some ((executorId, _)) =>
    +                  if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) {
    +                    blacklistTrackerOpt.foreach(blt => blt.killBlacklistedIdleExecutor(executorId))
    +
    +                    val timeout = conf.get(config.UNSCHEDULABLE_TASKSET_TIMEOUT) * 1000
    +                    unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis() + timeout
    +                    logInfo(s"Waiting for $timeout ms for completely "
    +                      + s"blacklisted task to be schedulable again before aborting $taskSet.")
    +                    abortTimer.schedule(
    +                      createUnschedulableTaskSetAbortTimer(taskSet, taskIndex), timeout)
    +                  }
    +                case _ => // Abort Immediately
    +                  logInfo("Cannot schedule any task because of complete blacklisting. No idle" +
    +                    s" executors can be found to kill. Aborting $taskSet." )
    +                  taskSet.abortSinceCompletelyBlacklisted(taskIndex)
    +              }
    +            case _ => // Do nothing if no tasks completely blacklisted.
    +          }
    +        } else {
    +          // We want to defer killing any taskSets as long as we have a non blacklisted executor
    +          // which can be used to schedule a task from any active taskSets. This ensures that the
    +          // job can make progress and if we encounter a flawed taskSet it will eventually either
    +          // fail or abort due to being completely blacklisted.
    --- End diff --
    
    ok, yeah it seems like it would have to be very timing dependent that taskset1 never got a chance for that executor, really that would just be a normal indefinite postponement problem in the scheduler regardless of blacklisting.   I don't think with fifo its a problem as first taskset should always be first. With Fair scheduler perhaps it could but probably depends on much more specific scenario.   
    
    I guess I'm ok with this if you are.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/4777/
    Test PASSed.


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r228642015
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +420,54 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case Some(taskIndex) => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we try to find an existing idle blacklisted
    +              // executor. If we cannot find one, we abort immediately. Else we kill the idle
    +              // executor and kick off an abortTimer which if it doesn't schedule a task within the
    +              // the timeout will abort the taskSet if we were unable to schedule any task from the
    +              // taskSet.
    +              // Note 1: We keep track of schedulability on a per taskSet basis rather than on a per
    +              // task basis.
    +              // Note 2: The taskSet can still be aborted when there are more than one idle
    +              // blacklisted executors and dynamic allocation is on. This can happen when a killed
    +              // idle executor isn't replaced in time by ExecutorAllocationManager as it relies on
    +              // pending tasks and doesn't kill executors on idle timeouts, resulting in the abort
    +              // timer to expire and abort the taskSet.
    +              executorIdToRunningTaskIds.find(x => !isExecutorBusy(x._1)) match {
    +                case Some ((executorId, _)) =>
    +                  if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) {
    +                    blacklistTrackerOpt.foreach(blt => blt.killBlacklistedIdleExecutor(executorId))
    +
    +                    val timeout = conf.get(config.UNSCHEDULABLE_TASKSET_TIMEOUT) * 1000
    +                    unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis() + timeout
    +                    logInfo(s"Waiting for $timeout ms for completely "
    +                      + s"blacklisted task to be schedulable again before aborting $taskSet.")
    +                    abortTimer.schedule(
    +                      createUnschedulableTaskSetAbortTimer(taskSet, taskIndex), timeout)
    +                  }
    +                case _ => // Abort Immediately
    +                  logInfo("Cannot schedule any task because of complete blacklisting. No idle" +
    +                    s" executors can be found to kill. Aborting $taskSet." )
    +                  taskSet.abortSinceCompletelyBlacklisted(taskIndex)
    +              }
    +            case _ => // Do nothing if no tasks completely blacklisted.
    --- End diff --
    
    doesn't matter a ton, I think its just a scala-ism it takes a while to get used to.  my rough guidline is: use pattern-matching if you're doing something distinct in both the Some and None cases, or if you can make use of more complex patterns to avoid more nesting (eg. `case Some(x) if x.isFoo() =>`).  If you're only doing something in the Some branch, then generally prefer map, foreach, filter, etc.
    
    My reason for wanting it here is that when I look at this code, I needed to scroll back to figure out what you were even matching on here and make sure you weren't ignoring something important.  When I see the `match` up above, I assume something is going to happen in both branches.  OTOH if there was a `foreach`, when I see the foreach I know right away you're ignoring None.
    
    again this is really minor, I don't actually care that much, just explaining my thinking.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/4063/
    Test PASSed.


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r227112966
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +420,55 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case Some(taskIndex) => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we try to find an existing idle blacklisted
    +              // executor. If we cannot find one, we abort immediately. Else we kill the idle
    +              // executor and kick off an abortTimer which if it doesn't schedule a task within the
    +              // the timeout will abort the taskSet if we were unable to schedule any task from the
    +              // taskSet.
    +              // Note 1: We keep track of schedulability on a per taskSet basis rather than on a per
    +              // task basis.
    +              // Note 2: The taskSet can still be aborted when there are more than one idle
    +              // blacklisted executors and dynamic allocation is on. This can happen when a killed
    +              // idle executor isn't replaced in time by ExecutorAllocationManager as it relies on
    +              // pending tasks and doesn't kill executors on idle timeouts, resulting in the abort
    +              // timer to expire and abort the taskSet.
    +              executorIdToRunningTaskIds.find(x => !isExecutorBusy(x._1)) match {
    --- End diff --
    
    sure, I thought the name `executorIdToRunningTaskIds` makes the other version clear enough, but I don't really feel strongly.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][Scheduler] Acquire new executors to avoid ...

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

    https://github.com/apache/spark/pull/22288
  
    @squito @tgravescs  Can you review this PR? Thanks.


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r222816923
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -414,9 +425,48 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    -        }
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case taskIndex: Some[Int] => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we kill an existing blacklisted executor/s and
    +              // kick off an abortTimer which after waiting will abort the taskSet if we were
    +              // unable to schedule any task from the taskSet.
    +              // Note: We keep a track of schedulability on a per taskSet basis rather than on a
    +              // per task basis.
    +              val executor = hostToExecutors.valuesIterator.next().iterator.next()
    +              logDebug("Killing executor because of task unschedulability: " + executor)
    +              blacklistTrackerOpt.foreach(blt => blt.killBlacklistedExecutor(executor))
    +
    +              if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) {
    +                  unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis()
    +                  abortTimer.schedule(new TimerTask() {
    +                    override def run() {
    +                      if (unschedulableTaskSetToExpiryTime.contains(taskSet) &&
    +                        (unschedulableTaskSetToExpiryTime(taskSet)
    +                          + UNSCHEDULABLE_TASKSET_TIMEOUT_MS)
    +                          <= clock.getTimeMillis()
    +                      ) {
    +                        logInfo("Cannot schedule any task because of complete blacklisting. " +
    +                          "Wait time for scheduling expired. Aborting the application.")
    +                        taskSet.abortSinceCompletelyBlacklisted(taskIndex.get)
    +                      } else {
    +                        this.cancel()
    +                      }
    +                    }
    +                  }, UNSCHEDULABLE_TASKSET_TIMEOUT_MS)
    +                }
    +            case _ => // Do nothing.
    +            }
    +          } else {
    +            // If a task was scheduled, we clear the expiry time for the taskSet. The abort timer
    +            // checks this entry to decide if we want to abort the taskSet.
    +            if (unschedulableTaskSetToExpiryTime.contains(taskSet)) {
    --- End diff --
    
    just calling the remove sounds like a good idea.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r228637254
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +420,54 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case Some(taskIndex) => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we try to find an existing idle blacklisted
    +              // executor. If we cannot find one, we abort immediately. Else we kill the idle
    +              // executor and kick off an abortTimer which if it doesn't schedule a task within the
    +              // the timeout will abort the taskSet if we were unable to schedule any task from the
    +              // taskSet.
    +              // Note 1: We keep track of schedulability on a per taskSet basis rather than on a per
    +              // task basis.
    +              // Note 2: The taskSet can still be aborted when there are more than one idle
    +              // blacklisted executors and dynamic allocation is on. This can happen when a killed
    +              // idle executor isn't replaced in time by ExecutorAllocationManager as it relies on
    +              // pending tasks and doesn't kill executors on idle timeouts, resulting in the abort
    +              // timer to expire and abort the taskSet.
    +              executorIdToRunningTaskIds.find(x => !isExecutorBusy(x._1)) match {
    +                case Some ((executorId, _)) =>
    +                  if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) {
    +                    blacklistTrackerOpt.foreach(blt => blt.killBlacklistedIdleExecutor(executorId))
    +
    +                    val timeout = conf.get(config.UNSCHEDULABLE_TASKSET_TIMEOUT) * 1000
    +                    unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis() + timeout
    +                    logInfo(s"Waiting for $timeout ms for completely "
    +                      + s"blacklisted task to be schedulable again before aborting $taskSet.")
    +                    abortTimer.schedule(
    +                      createUnschedulableTaskSetAbortTimer(taskSet, taskIndex), timeout)
    +                  }
    +                case _ => // Abort Immediately
    +                  logInfo("Cannot schedule any task because of complete blacklisting. No idle" +
    +                    s" executors can be found to kill. Aborting $taskSet." )
    +                  taskSet.abortSinceCompletelyBlacklisted(taskIndex)
    +              }
    +            case _ => // Do nothing if no tasks completely blacklisted.
    +          }
    +        } else {
    +          // We want to defer killing any taskSets as long as we have a non blacklisted executor
    +          // which can be used to schedule a task from any active taskSets. This ensures that the
    +          // job can make progress and if we encounter a flawed taskSet it will eventually either
    +          // fail or abort due to being completely blacklisted.
    --- End diff --
    
    Your understanding is correct. I will update the comment.


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r230682694
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala ---
    @@ -503,6 +507,182 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
         verify(tsm).abort(anyString(), anyObject())
       }
     
    +  test("SPARK-22148 abort timer should kick in when task is completely blacklisted & no new " +
    +      "executor can be acquired") {
    +    // set the abort timer to fail immediately
    +    taskScheduler = setupSchedulerWithMockTaskSetBlacklist(
    +      config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "0")
    +
    +    // We have only 1 task remaining with 1 executor
    +    val taskSet = FakeTask.createTaskSet(numTasks = 1, stageAttemptId = 0)
    +    taskScheduler.submitTasks(taskSet)
    +    val tsm = stageToMockTaskSetManager(0)
    +
    +    // submit an offer with one executor
    +    val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten
    +
    +    // Fail the running task
    +    val failedTask = firstTaskAttempts.find(_.executorId == "executor0").get
    +    taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0))
    +    // we explicitly call the handleFailedTask method here to avoid adding a sleep in the test suite
    +    // Reason being - handleFailedTask is run by an executor service and there is a momentary delay
    +    // before it is launched and this fails the assertion check.
    +    tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, UnknownReason)
    +    when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask(
    +      "executor0", failedTask.index)).thenReturn(true)
    +
    +    // make an offer on the blacklisted executor.  We won't schedule anything, and set the abort
    +    // timer to kick in immediately
    +    assert(taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten.size === 0)
    +    // Wait for the abort timer to kick in. Even though we configure the timeout to be 0, there is a
    +    // slight delay as the abort timer is launched in a separate thread.
    +    eventually(timeout(500.milliseconds)) {
    +      assert(tsm.isZombie)
    +    }
    +  }
    +
    +  test("SPARK-22148 try to acquire a new executor when task is unschedulable with 1 executor") {
    +    taskScheduler = setupSchedulerWithMockTaskSetBlacklist(
    +      config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "10")
    +
    +    // We have only 1 task remaining with 1 executor
    +    val taskSet = FakeTask.createTaskSet(numTasks = 1, stageAttemptId = 0)
    +    taskScheduler.submitTasks(taskSet)
    +    val tsm = stageToMockTaskSetManager(0)
    +
    +    // submit an offer with one executor
    +    val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten
    +
    +    // Fail the running task
    +    val failedTask = firstTaskAttempts.head
    +    taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0))
    +    // we explicitly call the handleFailedTask method here to avoid adding a sleep in the test suite
    +    // Reason being - handleFailedTask is run by an executor service and there is a momentary delay
    +    // before it is launched and this fails the assertion check.
    +    tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, UnknownReason)
    +    when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask(
    +      "executor0", failedTask.index)).thenReturn(true)
    +
    +    // make an offer on the blacklisted executor.  We won't schedule anything, and set the abort
    +    // timer to expire if no new executors could be acquired. We kill the existing idle blacklisted
    +    // executor and try to acquire a new one.
    +    assert(taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten.size === 0)
    +    assert(taskScheduler.unschedulableTaskSetToExpiryTime.contains(tsm))
    +    assert(!tsm.isZombie)
    +
    +    // Offer a new executor which should be accepted
    +    assert(taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor1", "host0", 1)
    +    )).flatten.size === 1)
    +    assert(taskScheduler.unschedulableTaskSetToExpiryTime.isEmpty)
    +    assert(!tsm.isZombie)
    +  }
    +
    +  // This is to test a scenario where we have two taskSets completely blacklisted and on acquiring
    +  // a new executor we don't want the abort timer for the second taskSet to expire and abort the job
    +  test("SPARK-22148 abort timer should clear unschedulableTaskSetToExpiryTime for all TaskSets") {
    +    taskScheduler = setupSchedulerWithMockTaskSetBlacklist()
    +
    +    // We have 2 taskSets with 1 task remaining in each with 1 executor completely blacklisted
    +    val taskSet1 = FakeTask.createTaskSet(numTasks = 1, stageId = 0, stageAttemptId = 0)
    +    taskScheduler.submitTasks(taskSet1)
    +    val taskSet2 = FakeTask.createTaskSet(numTasks = 1, stageId = 1, stageAttemptId = 0)
    +    taskScheduler.submitTasks(taskSet2)
    +    val tsm = stageToMockTaskSetManager(0)
    +
    +    // submit an offer with one executor
    +    val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten
    +
    +    assert(taskScheduler.unschedulableTaskSetToExpiryTime.isEmpty)
    +
    +    // Fail the running task
    +    val failedTask = firstTaskAttempts.head
    +    taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0))
    +    tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, UnknownReason)
    +    when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask(
    +      "executor0", failedTask.index)).thenReturn(true)
    +
    +    // make an offer. We will schedule the task from the second taskSet. Since a task was scheduled
    +    // we do not kick off the abort timer for taskSet1
    +    val secondTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten
    +
    +    assert(taskScheduler.unschedulableTaskSetToExpiryTime.isEmpty)
    +
    +    val tsm2 = stageToMockTaskSetManager(1)
    +    val failedTask2 = secondTaskAttempts.head
    +    taskScheduler.statusUpdate(failedTask2.taskId, TaskState.FAILED, ByteBuffer.allocate(0))
    +    tsm2.handleFailedTask(failedTask2.taskId, TaskState.FAILED, UnknownReason)
    +    when(tsm2.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask(
    +      "executor0", failedTask2.index)).thenReturn(true)
    +
    +    // make an offer on the blacklisted executor.  We won't schedule anything, and set the abort
    +    // timer for taskSet1 and taskSet2
    +    assert(taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten.size === 0)
    +    assert(taskScheduler.unschedulableTaskSetToExpiryTime.contains(tsm))
    +    assert(taskScheduler.unschedulableTaskSetToExpiryTime.contains(tsm2))
    +    assert(taskScheduler.unschedulableTaskSetToExpiryTime.size == 2)
    +
    +    // Offer a new executor which should be accepted
    +    assert(taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor1", "host1", 1)
    +    )).flatten.size === 1)
    +
    +    // Check if all the taskSets are cleared
    +    assert(taskScheduler.unschedulableTaskSetToExpiryTime.isEmpty)
    +
    +    assert(!tsm.isZombie)
    +  }
    +
    +  // this test is to check that we don't abort a taskSet which is not being scheduled on other
    +  // executors as it is waiting on locality timeout and not being aborted because it is still not
    +  // completely blacklisted.
    +  test("SPARK-22148 Ensure we don't abort the taskSet if we haven't been completely blacklisted") {
    +    taskScheduler = setupSchedulerWithMockTaskSetBlacklist(
    +      config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "0")
    +
    +    val preferredLocation = Seq(ExecutorCacheTaskLocation("host0", "executor0"))
    +    val taskSet1 = FakeTask.createTaskSet(numTasks = 1, stageId = 0, stageAttemptId = 0,
    +      preferredLocation)
    +    taskScheduler.submitTasks(taskSet1)
    +
    +    val tsm = stageToMockTaskSetManager(0)
    +
    +    // submit an offer with one executor
    +    var taskAttempts = taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten
    +
    +    // Fail the running task
    +    val failedTask = taskAttempts.head
    +    taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0))
    +    tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, UnknownReason)
    +    when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask(
    +      "executor0", failedTask.index)).thenReturn(true)
    +
    +    // make an offer but we won't schedule anything yet as scheduler locality is still PROCESS_LOCAL
    +    assert(taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor1", "host0", 1)
    +    )).flatten.isEmpty)
    --- End diff --
    
    this is dependent on the system clock not advancing past the locality timeout.  I've seen pauses on jenkins over 5 seconds in flaky tests -- either put in a manual clock or just increase the locality timeout in this test to avoid flakiness here


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #97943 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97943/testReport)** for PR 22288 at commit [`9b2aeaf`](https://github.com/apache/spark/commit/9b2aeaffdf6fc8d76b6c8ba2978c5dc7d6022899).


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark pull request #22288: [SPARK-22148][Scheduler] Acquire new executors to...

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

    https://github.com/apache/spark/pull/22288#discussion_r214719743
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -414,9 +425,54 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    -        }
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case taskIndex: Some[Int] => // Returns the taskIndex which was unschedulable
    +              if (conf.getBoolean("spark.dynamicAllocation.enabled", false)) {
    +                // If the taskSet is unschedulable we kill the existing blacklisted executor/s and
    +                // kick off an abortTimer which after waiting will abort the taskSet if we were
    +                // unable to get new executors and couldn't schedule a task from the taskSet.
    +                // Note: We keep a track of schedulability on a per taskSet basis rather than on a
    +                // per task basis.
    +                if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) {
    +                  hostToExecutors.valuesIterator.foreach(executors => executors.foreach({
    +                    executor =>
    +                      logDebug("Killing executor because of task unschedulability: " + executor)
    +                      blacklistTrackerOpt.foreach(blt => blt.killBlacklistedExecutor(executor))
    --- End diff --
    
    Seriously? You killed all executors ? What if other taskSets' tasks are running on them ?
    
    BTW, if you want to refresh executors, you have to enable `spark.blacklist.killBlacklistedExecutors`  also.


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r228604552
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +420,54 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case Some(taskIndex) => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we try to find an existing idle blacklisted
    +              // executor. If we cannot find one, we abort immediately. Else we kill the idle
    +              // executor and kick off an abortTimer which if it doesn't schedule a task within the
    +              // the timeout will abort the taskSet if we were unable to schedule any task from the
    +              // taskSet.
    +              // Note 1: We keep track of schedulability on a per taskSet basis rather than on a per
    +              // task basis.
    +              // Note 2: The taskSet can still be aborted when there are more than one idle
    +              // blacklisted executors and dynamic allocation is on. This can happen when a killed
    +              // idle executor isn't replaced in time by ExecutorAllocationManager as it relies on
    +              // pending tasks and doesn't kill executors on idle timeouts, resulting in the abort
    +              // timer to expire and abort the taskSet.
    +              executorIdToRunningTaskIds.find(x => !isExecutorBusy(x._1)) match {
    +                case Some ((executorId, _)) =>
    +                  if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) {
    +                    blacklistTrackerOpt.foreach(blt => blt.killBlacklistedIdleExecutor(executorId))
    +
    +                    val timeout = conf.get(config.UNSCHEDULABLE_TASKSET_TIMEOUT) * 1000
    +                    unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis() + timeout
    +                    logInfo(s"Waiting for $timeout ms for completely "
    +                      + s"blacklisted task to be schedulable again before aborting $taskSet.")
    +                    abortTimer.schedule(
    +                      createUnschedulableTaskSetAbortTimer(taskSet, taskIndex), timeout)
    +                  }
    +                case _ => // Abort Immediately
    --- End diff --
    
    really minor, I think its a bit more clear if you say `case None` here (otherwise I take a just a second to figure out what other patterns will fall under this catch-all)


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r226999715
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +420,54 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case Some(taskIndex) => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we try to find an existing idle blacklisted
    +              // executor. If we cannot find one, we abort immediately. Else we kill the idle
    +              // executor and kick off an abortTimer which if it doesn't schedule a task within the
    +              // the timeout will abort the taskSet if we were unable to schedule any task from the
    +              // taskSet.
    +              // Note 1: We keep track of schedulability on a per taskSet basis rather than on a per
    +              // task basis.
    +              // Note 2: The taskSet can still be aborted when there are more than one idle
    +              // blacklisted executors and dynamic allocation is on. This can happen when a killed
    +              // idle executor isn't replaced in time by ExecutorAllocationManager as it relies on
    +              // pending tasks and doesn't kill executors on idle timeouts, resulting in the abort
    +              // timer to expire and abort the taskSet.
    +              executorIdToRunningTaskIds.find(x => !isExecutorBusy(x._1)) match {
    +                case Some (x) =>
    +                  val executorId = x._1
    +                  if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) {
    +                    blacklistTrackerOpt.foreach(blt => blt.killBlacklistedIdleExecutor(executorId))
    +
    +                    unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis()
    +                    val timeout = conf.get(config.UNSCHEDULABLE_TASKSET_TIMEOUT) * 1000
    +                    logInfo(s"Waiting for $timeout ms for completely "
    +                      + s"blacklisted task to be schedulable again before aborting $taskSet.")
    +                    abortTimer.schedule(getAbortTimer(taskSet, taskIndex, timeout), timeout)
    +                  }
    +                case _ => // Abort Immediately
    +                  logInfo("Cannot schedule any task because of complete blacklisting. No idle" +
    +                    s" executors can be found to kill. Aborting $taskSet." )
    +                  taskSet.abortSinceCompletelyBlacklisted(taskIndex)
    +              }
    +            case _ => // Do nothing if no tasks completely blacklisted.
    +          }
    +        } else {
    +          // We want to differ killing any taskSets as long as we have a non blacklisted executor
    --- End diff --
    
    s/differ/defer/


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/3354/
    Test PASSed.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #97850 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97850/testReport)** for PR 22288 at commit [`b2d0d40`](https://github.com/apache/spark/commit/b2d0d40771534291bdd5a1e3ebfc2c0c227c5956).


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    Build finished. Test FAILed.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    retest this please


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/4418/
    Test PASSed.


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r227061320
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala ---
    @@ -82,10 +84,12 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
         setupHelper()
       }
     
    -  def setupSchedulerWithMockTaskSetBlacklist(): TaskSchedulerImpl = {
    +  def setupSchedulerWithMockTaskSetBlacklist(confs: (String, String)*): TaskSchedulerImpl = {
         blacklist = mock[BlacklistTracker]
         val conf = new SparkConf().setMaster("local").setAppName("TaskSchedulerImplSuite")
         conf.set(config.BLACKLIST_ENABLED, true)
    +    confs.foreach{ case (k, v) => conf.set(k, v) }
    --- End diff --
    
    super nit: space before `{`


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r228605235
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +420,54 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case Some(taskIndex) => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we try to find an existing idle blacklisted
    +              // executor. If we cannot find one, we abort immediately. Else we kill the idle
    +              // executor and kick off an abortTimer which if it doesn't schedule a task within the
    +              // the timeout will abort the taskSet if we were unable to schedule any task from the
    +              // taskSet.
    +              // Note 1: We keep track of schedulability on a per taskSet basis rather than on a per
    +              // task basis.
    +              // Note 2: The taskSet can still be aborted when there are more than one idle
    +              // blacklisted executors and dynamic allocation is on. This can happen when a killed
    +              // idle executor isn't replaced in time by ExecutorAllocationManager as it relies on
    +              // pending tasks and doesn't kill executors on idle timeouts, resulting in the abort
    +              // timer to expire and abort the taskSet.
    +              executorIdToRunningTaskIds.find(x => !isExecutorBusy(x._1)) match {
    +                case Some ((executorId, _)) =>
    +                  if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) {
    +                    blacklistTrackerOpt.foreach(blt => blt.killBlacklistedIdleExecutor(executorId))
    +
    +                    val timeout = conf.get(config.UNSCHEDULABLE_TASKSET_TIMEOUT) * 1000
    +                    unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis() + timeout
    +                    logInfo(s"Waiting for $timeout ms for completely "
    +                      + s"blacklisted task to be schedulable again before aborting $taskSet.")
    +                    abortTimer.schedule(
    +                      createUnschedulableTaskSetAbortTimer(taskSet, taskIndex), timeout)
    +                  }
    +                case _ => // Abort Immediately
    +                  logInfo("Cannot schedule any task because of complete blacklisting. No idle" +
    +                    s" executors can be found to kill. Aborting $taskSet." )
    +                  taskSet.abortSinceCompletelyBlacklisted(taskIndex)
    +              }
    +            case _ => // Do nothing if no tasks completely blacklisted.
    +          }
    +        } else {
    +          // We want to defer killing any taskSets as long as we have a non blacklisted executor
    +          // which can be used to schedule a task from any active taskSets. This ensures that the
    +          // job can make progress and if we encounter a flawed taskSet it will eventually either
    +          // fail or abort due to being completely blacklisted.
    --- End diff --
    
    I think you should say here that you may have a job wait indefinitely, if its effectively blacklisted the entire cluster, but other jobs keep coming in and keeping resources occupied so the cluster stays busy.  So its not really accurate to say that it will be aborted eventually, we are actually *not* guaranteeing that (if I understood things correctly).
    
    Since its folded now lemme reference the prior discussion on this: https://github.com/apache/spark/pull/22288#discussion_r226477452
    
    >> Want to make sure I understand this part, and why you aren't only clearing the timer for the taskset you just scheduled a task for. If you have multiple tasksets running simultaneously, one is making progress but the other is totally blacklisted, I guess you do not want to kill anything, because that would mess with the taskset that is working correctly? Instead you'll just let the taskset which is totally blacklisted eventually fail from the timeout? I guess that makes sense, because if one taskset is progressing, it means the failing taskset probably is probably flawed, not the executors.
    >>
    >>If that's right, would be good to include something along those lines in the comment (personally I don't find a comment about how its related to the timer that useful, that's obvious from the code).
    >
     >dhruve 7 days ago  Contributor
    >That is correct. It also covers other scenario that @tgravescs originally pointed out.
    >
    >Lets say if you have multiple taskSets running which are completely blacklisted. If you were able to get an executor, you would just clear the timer for that specific taskSet. Now due to resource constraint, if you weren't able to obtain another executor within the timeout for the other taskSet, you would abort the other taskSet when you could actually wait for it to be scheduled on the newly obtained executor.
    >
    > So clearing the timer for all the taskSets ensures that currently we aren't in a completely blacklisted state and should try to run to completion. However if the taskset itself is flawed, we would eventually fail. This could result in wasted effort, but we don't have a way to determine that yet, so this should be okay.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r225193634
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +419,61 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case taskIndex: Some[Int] => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we try to find an existing idle blacklisted
    +              // executor. If we cannot find one, we abort immediately. Else we kill the idle
    +              // executor and kick off an abortTimer which if it doesn't schedule a task within the
    +              // the timeout will abort the taskSet if we were unable to schedule any task from the
    +              // taskSet.
    +              // Note 1: We keep track of schedulability on a per taskSet basis rather than on a per
    +              // task basis.
    +              // Note 2: The taskSet can still be aborted when there are more than one idle
    +              // blacklisted executors and dynamic allocation is on. This can happen when a killed
    +              // idle executor isn't replaced in time by ExecutorAllocationManager as it relies on
    +              // pending tasks and doesn't kill executors on idle timeouts, resulting in the abort
    +              // timer to expire and abort the taskSet.
    +              executorIdToRunningTaskIds.find(x => !isExecutorBusy(x._1)) match {
    +                case Some (x) =>
    +                  val executorId = x._1
    +                  if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) {
    +                    blacklistTrackerOpt.foreach(blt => blt.killBlacklistedIdleExecutor(executorId))
    +
    +                    unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis()
    +                    val timeout = conf.get(config.UNSCHEDULABLE_TASKSET_TIMEOUT) * 1000
    +                    logInfo(s"Waiting for $timeout ms for completely "
    +                      + s"blacklisted task to be schedulable again before aborting $taskSet.")
    +                    abortTimer.schedule(new TimerTask() {
    +                      override def run() {
    +                        if (unschedulableTaskSetToExpiryTime.contains(taskSet) &&
    +                          (unschedulableTaskSetToExpiryTime(taskSet) + timeout)
    +                            <= clock.getTimeMillis()
    +                        ) {
    +                          logInfo("Cannot schedule any task because of complete blacklisting. " +
    +                            s"Wait time for scheduling expired. Aborting $taskSet.")
    +                          taskSet.abortSinceCompletelyBlacklisted(taskIndex.get)
    +                        } else {
    +                          this.cancel()
    +                        }
    +                      }
    +                    }, timeout)
    +                  }
    +                case _ => // Abort Immediately
    +                  logInfo("Cannot schedule any task because of complete blacklisting. No idle" +
    +                  s" executors can be found to kill. Aborting $taskSet." )
    +                  taskSet.abortSinceCompletelyBlacklisted(taskIndex.get)
    +              }
    +            case _ => // Do nothing if no tasks completely blacklisted.
    +          }
    +        } else {
    +          // If a task was scheduled, we clear the expiry time for the taskSet. The abort timer
    +          // checks this entry to decide if we want to abort the taskSet.
    +          unschedulableTaskSetToExpiryTime.remove(taskSet)
    --- End diff --
    
    Here we have to handle the situation where if you have 2 tasksets, they may have both chose the same executor to kill.  If one of the tasksets kills the executor and launches a task it clears it expiry, but if a second taskset had tried to kill the same executor we don't clear it and it could end up aborting the second taskset and killing the job even though it shouldn't have


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r227005479
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala ---
    @@ -503,6 +505,145 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
         verify(tsm).abort(anyString(), anyObject())
       }
     
    +  test("SPARK-22148 abort timer should kick in when task is completely blacklisted & no new " +
    +    "executor can be acquired") {
    +    // set the abort timer to fail immediately
    +    taskScheduler = setupSchedulerWithMockTaskSetBlacklist(
    +      config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "0")
    +
    +    // We have only 1 task remaining with 1 executor
    +    val taskSet = FakeTask.createTaskSet(numTasks = 1, stageAttemptId = 0)
    +    taskScheduler.submitTasks(taskSet)
    +    val tsm = stageToMockTaskSetManager(0)
    +
    +    // submit an offer with one executor
    +    val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten
    +
    +    // Fail the running task
    +    val failedTask = firstTaskAttempts.find(_.executorId == "executor0").get
    +    taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0))
    +    // we explicitly call the handleFailedTask method here to avoid adding a sleep in the test suite
    +    // Reason being - handleFailedTask is run by an executor service and there is a momentary delay
    +    // before it is launched and this fails the assertion check.
    +    tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, UnknownReason)
    +    when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask(
    +      "executor0", failedTask.index)).thenReturn(true)
    +
    +    // make an offer on the blacklisted executor.  We won't schedule anything, and set the abort
    +    // timer to kick in immediately
    +    assert(taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten.size === 0)
    +    // Wait for the abort timer to kick in. Without sleep the test exits before the timer is
    +    // triggered.
    +    Thread.sleep(500)
    --- End diff --
    
    instead of sleep could you do something like, note I haven't used eventually and the scaladoc seems to be down but its used in other places like SparkContextSuite
    
    eventually(timeout(1.seconds)) {
                assert(tsm.isZombie)
              }


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r222783407
  
    --- Diff: core/src/main/scala/org/apache/spark/internal/config/package.scala ---
    @@ -597,6 +597,17 @@ package object config {
           .checkValue(v => v > 0, "The value should be a positive time value.")
           .createWithDefaultString("365d")
     
    +  // Threshold above which we abort the TaskSet if a task could not be scheduled because of complete
    +  // blacklisting.
    +  private[spark] val UNSCHEDULABLE_TASKSET_TIMEOUT =
    +    ConfigBuilder("spark.scheduler.unschedulableTaskSetTimeout")
    +      .doc("The timeout in seconds to wait before aborting a TaskSet to acquire a new executor " +
    --- End diff --
    
    reword to be timeout in seconds to wait to try to acquire a new executor and schedule a task before aborting....  


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #97622 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97622/testReport)** for PR 22288 at commit [`b2d0d40`](https://github.com/apache/spark/commit/b2d0d40771534291bdd5a1e3ebfc2c0c227c5956).


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/4045/
    Test PASSed.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #96432 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/96432/testReport)** for PR 22288 at commit [`ffbc9c3`](https://github.com/apache/spark/commit/ffbc9c32d14a0c82036defb90eb18167f93bad4d).


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/4374/
    Test PASSed.


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r227062956
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala ---
    @@ -503,6 +507,145 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
         verify(tsm).abort(anyString(), anyObject())
       }
     
    +  test("SPARK-22148 abort timer should kick in when task is completely blacklisted & no new " +
    +    "executor can be acquired") {
    +    // set the abort timer to fail immediately
    +    taskScheduler = setupSchedulerWithMockTaskSetBlacklist(
    +      config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "0")
    +
    +    // We have only 1 task remaining with 1 executor
    +    val taskSet = FakeTask.createTaskSet(numTasks = 1, stageAttemptId = 0)
    +    taskScheduler.submitTasks(taskSet)
    +    val tsm = stageToMockTaskSetManager(0)
    +
    +    // submit an offer with one executor
    +    val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten
    +
    +    // Fail the running task
    +    val failedTask = firstTaskAttempts.find(_.executorId == "executor0").get
    +    taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0))
    +    // we explicitly call the handleFailedTask method here to avoid adding a sleep in the test suite
    +    // Reason being - handleFailedTask is run by an executor service and there is a momentary delay
    +    // before it is launched and this fails the assertion check.
    +    tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, UnknownReason)
    +    when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask(
    +      "executor0", failedTask.index)).thenReturn(true)
    +
    +    // make an offer on the blacklisted executor.  We won't schedule anything, and set the abort
    +    // timer to kick in immediately
    +    assert(taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten.size === 0)
    +    // Wait for the abort timer to kick in. Without sleep the test exits before the timer is
    +    // triggered.
    +    eventually(timeout(500.milliseconds)) {
    +      assert(tsm.isZombie)
    +    }
    +  }
    +
    +  test("SPARK-22148 try to acquire a new executor when task is unschedulable with 1 executor") {
    +    taskScheduler = setupSchedulerWithMockTaskSetBlacklist(
    +      config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "10")
    +
    +    // We have only 1 task remaining with 1 executor
    +    val taskSet = FakeTask.createTaskSet(numTasks = 1, stageAttemptId = 0)
    +    taskScheduler.submitTasks(taskSet)
    +    val tsm = stageToMockTaskSetManager(0)
    +
    +    // submit an offer with one executor
    +    val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten
    +
    +    // Fail the running task
    +    val failedTask = firstTaskAttempts.find(_.executorId == "executor0").get
    +    taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0))
    +    // we explicitly call the handleFailedTask method here to avoid adding a sleep in the test suite
    +    // Reason being - handleFailedTask is run by an executor service and there is a momentary delay
    +    // before it is launched and this fails the assertion check.
    +    tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, UnknownReason)
    +    when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask(
    +      "executor0", failedTask.index)).thenReturn(true)
    +
    +    // make an offer on the blacklisted executor.  We won't schedule anything, and set the abort
    +    // timer to expire if no new executors could be acquired. We kill the existing idle blacklisted
    +    // executor and try to acquire a new one.
    +    assert(taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten.size === 0)
    +    assert(!tsm.isZombie)
    +
    +    // Offer a new executor which should be accepted
    +    assert(taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor1", "host0", 1)
    +    )).flatten.size === 1)
    --- End diff --
    
    you need this to happen within 10 millis of the offer just before this, right?  If so, this is going to lead to a lot of flakiness in the tests, as occasionally there are long pauses from gc etc. unrelated to the test.  I think you need to have a manual clock in the `TaskScheduleImpl` for tests.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #98107 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98107/testReport)** for PR 22288 at commit [`aac1e9e`](https://github.com/apache/spark/commit/aac1e9ee1ed5935c6370556b0ade4be3a5db4f29).


---

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


[GitHub] spark issue #22288: [SPARK-22148][Scheduler] Acquire new executors to avoid ...

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

    https://github.com/apache/spark/pull/22288
  
    Ok I looked at jiras, and this looks it also covers SPARK-15815, right? you could add that to the summary too. 
    
    You mention some future improvements:
    > Taking into account static allocation
    
    I mentioned this on an inline comment too, but now that I'm thinking about, it seems like this will be fine with static allocation as well.  It just seems like the problem is the worst in DA, as you can end up with one executor left for the straggler task, and then that executor gets blacklisted.  But, with static allocation, maybe you only requested a small number of executors on a large cluster, and by chance you get them all on a host with bad disks, so then everything starts failing.  You could still just kill those executors and request new ones to keep things going.  Anything I'm missing?
    
    > Querying the RM to figure out if its a small cluster, then try to wait some more or abort immediately.
    
    what's the concern here -- that if you're on a small cluster, there is very little chance of getting a good replacement so you should go back to failing fast?  I guess that would be nice, but much less important in my opinion.
    
    > Try to distinguish between waiting for time while you acquire an executor and time for being unable to schedule a task.
    
    I don't understand this part -- do you mean for locality preferences?


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #97165 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97165/testReport)** for PR 22288 at commit [`43e0af2`](https://github.com/apache/spark/commit/43e0af2238855dc69c93b36e40493c3eda670e90).


---

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


[GitHub] spark issue #22288: [SPARK-22148][Scheduler] Acquire new executors to avoid ...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #95485 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95485/testReport)** for PR 22288 at commit [`5253b31`](https://github.com/apache/spark/commit/5253b3134119b2a28cdaa1406d7bafb55f62cbc1).


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #97171 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97171/testReport)** for PR 22288 at commit [`a12a3fb`](https://github.com/apache/spark/commit/a12a3fbc5a40e8182a8029d5a38b347671f3c26e).


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #97499 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97499/testReport)** for PR 22288 at commit [`69c156b`](https://github.com/apache/spark/commit/69c156b32cf573c7d3a75f63ab093022046ad856).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r227067905
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +420,55 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case Some(taskIndex) => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we try to find an existing idle blacklisted
    +              // executor. If we cannot find one, we abort immediately. Else we kill the idle
    --- End diff --
    
    I'm a little worried that the idle condition will be too strict in some scenarios, if there is a large backlog of tasks from another taskset, or whatever the error is, the tasks take a while to fail (eg., you've really got a bad executor, but its not apparent till after network timeouts or something).  Eg. that could happen if you're doing a big join, and while preparing the input on the map-side, one side just has one straggler left but the other side still has a big backlog of tasks.  Or, in a jobserver style situation, and there are always other tasksets coming in.
    
    that said, I don't have any better ideas at the moment, and I still think this is an improvement.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #97625 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97625/testReport)** for PR 22288 at commit [`b2d0d40`](https://github.com/apache/spark/commit/b2d0d40771534291bdd5a1e3ebfc2c0c227c5956).


---

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


[GitHub] spark pull request #22288: [SPARK-22148][Scheduler] Acquire new executors to...

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

    https://github.com/apache/spark/pull/22288#discussion_r214720097
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -414,9 +425,54 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    -        }
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case taskIndex: Some[Int] => // Returns the taskIndex which was unschedulable
    +              if (conf.getBoolean("spark.dynamicAllocation.enabled", false)) {
    +                // If the taskSet is unschedulable we kill the existing blacklisted executor/s and
    +                // kick off an abortTimer which after waiting will abort the taskSet if we were
    +                // unable to get new executors and couldn't schedule a task from the taskSet.
    +                // Note: We keep a track of schedulability on a per taskSet basis rather than on a
    +                // per task basis.
    +                if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) {
    +                  hostToExecutors.valuesIterator.foreach(executors => executors.foreach({
    +                    executor =>
    +                      logDebug("Killing executor because of task unschedulability: " + executor)
    +                      blacklistTrackerOpt.foreach(blt => blt.killBlacklistedExecutor(executor))
    +                  })
    +                  )
    +                  unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis()
    +                  abortTimer.schedule(new TimerTask() {
    +                    override def run() {
    +                      if (unschedulableTaskSetToExpiryTime.contains(taskSet) &&
    +                        (unschedulableTaskSetToExpiryTime(taskSet)
    +                          + UNSCHEDULABLE_TASKSET_TIMEOUT_MS)
    +                          <= clock.getTimeMillis()
    +                      ) {
    +                        logInfo("Cannot schedule any task because of complete blacklisting. " +
    +                          "Wait time for scheduling expired. Aborting the application.")
    +                        taskSet.abortSinceCompletelyBlacklisted(taskIndex.get)
    +                      } else {
    +                        this.cancel()
    +                      }
    +                    }
    +                  }, UNSCHEDULABLE_TASKSET_TIMEOUT_MS)
    +                }
    +              } else {
    +                // TODO: try acquiring new executors for static allocation before aborting.
    --- End diff --
    
    How ? Waiting for other tasks finish and release resources ?


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r230487066
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala ---
    @@ -503,6 +507,181 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
         verify(tsm).abort(anyString(), anyObject())
       }
     
    +  test("SPARK-22148 abort timer should kick in when task is completely blacklisted & no new " +
    +      "executor can be acquired") {
    +    // set the abort timer to fail immediately
    +    taskScheduler = setupSchedulerWithMockTaskSetBlacklist(
    +      config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "0")
    +
    +    // We have only 1 task remaining with 1 executor
    +    val taskSet = FakeTask.createTaskSet(numTasks = 1, stageAttemptId = 0)
    +    taskScheduler.submitTasks(taskSet)
    +    val tsm = stageToMockTaskSetManager(0)
    +
    +    // submit an offer with one executor
    +    val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten
    +
    +    // Fail the running task
    +    val failedTask = firstTaskAttempts.find(_.executorId == "executor0").get
    +    taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0))
    +    // we explicitly call the handleFailedTask method here to avoid adding a sleep in the test suite
    +    // Reason being - handleFailedTask is run by an executor service and there is a momentary delay
    +    // before it is launched and this fails the assertion check.
    +    tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, UnknownReason)
    +    when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask(
    +      "executor0", failedTask.index)).thenReturn(true)
    +
    +    // make an offer on the blacklisted executor.  We won't schedule anything, and set the abort
    +    // timer to kick in immediately
    +    assert(taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten.size === 0)
    +    // Wait for the abort timer to kick in. Without sleep the test exits before the timer is
    +    // triggered.
    --- End diff --
    
    Comment still out of date


---

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


[GitHub] spark issue #22288: [SPARK-22148][Scheduler] Acquire new executors to avoid ...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #95485 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95485/testReport)** for PR 22288 at commit [`5253b31`](https://github.com/apache/spark/commit/5253b3134119b2a28cdaa1406d7bafb55f62cbc1).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #97469 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97469/testReport)** for PR 22288 at commit [`2c5a753`](https://github.com/apache/spark/commit/2c5a75354d36d08199b9805a7513a4ec4a546a27).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r228675680
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +420,54 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case Some(taskIndex) => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we try to find an existing idle blacklisted
    +              // executor. If we cannot find one, we abort immediately. Else we kill the idle
    +              // executor and kick off an abortTimer which if it doesn't schedule a task within the
    +              // the timeout will abort the taskSet if we were unable to schedule any task from the
    +              // taskSet.
    +              // Note 1: We keep track of schedulability on a per taskSet basis rather than on a per
    +              // task basis.
    +              // Note 2: The taskSet can still be aborted when there are more than one idle
    +              // blacklisted executors and dynamic allocation is on. This can happen when a killed
    +              // idle executor isn't replaced in time by ExecutorAllocationManager as it relies on
    +              // pending tasks and doesn't kill executors on idle timeouts, resulting in the abort
    +              // timer to expire and abort the taskSet.
    +              executorIdToRunningTaskIds.find(x => !isExecutorBusy(x._1)) match {
    +                case Some ((executorId, _)) =>
    +                  if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) {
    +                    blacklistTrackerOpt.foreach(blt => blt.killBlacklistedIdleExecutor(executorId))
    +
    +                    val timeout = conf.get(config.UNSCHEDULABLE_TASKSET_TIMEOUT) * 1000
    +                    unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis() + timeout
    +                    logInfo(s"Waiting for $timeout ms for completely "
    +                      + s"blacklisted task to be schedulable again before aborting $taskSet.")
    +                    abortTimer.schedule(
    +                      createUnschedulableTaskSetAbortTimer(taskSet, taskIndex), timeout)
    +                  }
    +                case _ => // Abort Immediately
    +                  logInfo("Cannot schedule any task because of complete blacklisting. No idle" +
    +                    s" executors can be found to kill. Aborting $taskSet." )
    +                  taskSet.abortSinceCompletelyBlacklisted(taskIndex)
    +              }
    +            case _ => // Do nothing if no tasks completely blacklisted.
    +          }
    +        } else {
    +          // We want to defer killing any taskSets as long as we have a non blacklisted executor
    +          // which can be used to schedule a task from any active taskSets. This ensures that the
    +          // job can make progress and if we encounter a flawed taskSet it will eventually either
    +          // fail or abort due to being completely blacklisted.
    --- End diff --
    
    Here's the scenario I'm worried about:
    
    1) taskset1 and taskset2 are both running currently.  taskset1 has enough failures to get blacklisted everywhere.
    2) there is an idle executor, even though taskset2 is running (eg. the executor that is available doesn't meet the locality preferences of taskset2).  So abort timer is started.
    3) the idle executor is killed, and you get a new one.
    4) just by luck, taskset2 gets a hold of the new idle executor (eg. the executor is on a node blacklisted by taskset1, or taskset2 just has a higher priority).  abort timer is cleared
    5) taskset2 finishes, but meanwhile taskset3 has been launched, and it uses the idle executor.   etc. for taskSetN, so you keep launching tasks, abort timer gets cleared, but nothing even gets scheduled on taskset1.
    
    admittedly this would not be the normal scenario -- you'll need more tasksets to keep coming, and you need tight enough resource constraints that taskset1 never get a hold of anything, even the new one.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    merged to master and 2.4 branch, thanks @dhruve 


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #96767 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/96767/testReport)** for PR 22288 at commit [`4c88168`](https://github.com/apache/spark/commit/4c881680fdde32244030b54b44125ac217dacb0d).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22288: [SPARK-22148][Scheduler] Acquire new executors to avoid ...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #95729 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95729/testReport)** for PR 22288 at commit [`87c4e57`](https://github.com/apache/spark/commit/87c4e57bb966078c8a78eabc5a5e4b6f60c78f28).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r216795021
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -414,9 +425,48 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    -        }
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case taskIndex: Some[Int] => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we kill an existing blacklisted executor/s and
    +              // kick off an abortTimer which after waiting will abort the taskSet if we were
    +              // unable to schedule any task from the taskSet.
    +              // Note: We keep a track of schedulability on a per taskSet basis rather than on a
    +              // per task basis.
    +              val executor = hostToExecutors.valuesIterator.next().iterator.next()
    --- End diff --
    
    That's a nice suggestion. 
    
    There was a case where you could have a few executors running, let's say just 3 of them and all are blacklisted but have some tasks running on them. To satisfy this,  I had started modifying this to take down an executor with the least no. of tasks running on them. I'll check some more on this.


---

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


[GitHub] spark issue #22288: [SPARK-22148][Scheduler] Acquire new executors to avoid ...

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

    https://github.com/apache/spark/pull/22288
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/2883/
    Test PASSed.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #96432 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/96432/testReport)** for PR 22288 at commit [`ffbc9c3`](https://github.com/apache/spark/commit/ffbc9c32d14a0c82036defb90eb18167f93bad4d).
     * This patch **fails from timeout after a configured wait of \`300m\`**.
     * This patch **does not merge cleanly**.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #98481 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98481/testReport)** for PR 22288 at commit [`a30276f`](https://github.com/apache/spark/commit/a30276f3780476ac39de766deef914de8b7b7e0a).


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r228606711
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala ---
    @@ -503,6 +507,181 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
         verify(tsm).abort(anyString(), anyObject())
       }
     
    +  test("SPARK-22148 abort timer should kick in when task is completely blacklisted & no new " +
    +      "executor can be acquired") {
    +    // set the abort timer to fail immediately
    +    taskScheduler = setupSchedulerWithMockTaskSetBlacklist(
    +      config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "0")
    +
    +    // We have only 1 task remaining with 1 executor
    +    val taskSet = FakeTask.createTaskSet(numTasks = 1, stageAttemptId = 0)
    +    taskScheduler.submitTasks(taskSet)
    +    val tsm = stageToMockTaskSetManager(0)
    +
    +    // submit an offer with one executor
    +    val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten
    +
    +    // Fail the running task
    +    val failedTask = firstTaskAttempts.find(_.executorId == "executor0").get
    +    taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0))
    +    // we explicitly call the handleFailedTask method here to avoid adding a sleep in the test suite
    +    // Reason being - handleFailedTask is run by an executor service and there is a momentary delay
    +    // before it is launched and this fails the assertion check.
    +    tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, UnknownReason)
    +    when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask(
    +      "executor0", failedTask.index)).thenReturn(true)
    +
    +    // make an offer on the blacklisted executor.  We won't schedule anything, and set the abort
    +    // timer to kick in immediately
    +    assert(taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten.size === 0)
    +    // Wait for the abort timer to kick in. Without sleep the test exits before the timer is
    +    // triggered.
    --- End diff --
    
    comment is out of date, there is no sleep anymore.  But it is still worth explaining that even though there is no configured delay, we still have to wait for the abort timer to run in a separate thread.


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r222810004
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +421,63 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    -        }
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case taskIndex: Some[Int] => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we try to find an existing idle blacklisted
    +              // executor. If we cannot find one, we abort immediately. Else we kill the idle
    +              // executor and kick off an abortTimer which after waiting will abort the taskSet if
    --- End diff --
    
    which if it doesn't schedule a task within the timeout will abort the taskset


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r224879925
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -146,21 +146,31 @@ private[scheduler] class BlacklistTracker (
         nextExpiryTime = math.min(execMinExpiry, nodeMinExpiry)
       }
     
    +  private def killExecutor(exec: String, msg: String): Unit = {
    +    allocationClient match {
    +      case Some(a) =>
    +        logInfo(msg)
    +        a.killExecutors(Seq(exec), adjustTargetNumExecutors = false, countFailures = false,
    +          force = true)
    +      case None =>
    +        logInfo(s"Not attempting to kill blacklisted executor id $exec " +
    +          s"since allocation client is not defined.")
    +    }
    +  }
    +
       private def killBlacklistedExecutor(exec: String): Unit = {
         if (conf.get(config.BLACKLIST_KILL_ENABLED)) {
    -      allocationClient match {
    -        case Some(a) =>
    -          logInfo(s"Killing blacklisted executor id $exec " +
    -            s"since ${config.BLACKLIST_KILL_ENABLED.key} is set.")
    -          a.killExecutors(Seq(exec), adjustTargetNumExecutors = false, countFailures = false,
    -            force = true)
    -        case None =>
    -          logWarning(s"Not attempting to kill blacklisted executor id $exec " +
    -            s"since allocation client is not defined.")
    -      }
    +      killExecutor(exec,
    +        s"Killing blacklisted executor id $exec since ${config.BLACKLIST_KILL_ENABLED.key} is set.")
         }
       }
     
    +  private[scheduler] def killBlacklistedIdleExecutor(exec: String): Unit = {
    +    killExecutor(exec,
    --- End diff --
    
    Makes sense. I guess there is no point in toggling this on and off (e.g. no IDLE_BLACKLIST_KILL_ENABLED)


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    cc @jiangxb1987 @attilapiros also for thoughts


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r227064894
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala ---
    @@ -503,6 +507,145 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
         verify(tsm).abort(anyString(), anyObject())
       }
     
    +  test("SPARK-22148 abort timer should kick in when task is completely blacklisted & no new " +
    +    "executor can be acquired") {
    +    // set the abort timer to fail immediately
    +    taskScheduler = setupSchedulerWithMockTaskSetBlacklist(
    +      config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "0")
    +
    +    // We have only 1 task remaining with 1 executor
    +    val taskSet = FakeTask.createTaskSet(numTasks = 1, stageAttemptId = 0)
    +    taskScheduler.submitTasks(taskSet)
    +    val tsm = stageToMockTaskSetManager(0)
    +
    +    // submit an offer with one executor
    +    val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten
    +
    +    // Fail the running task
    +    val failedTask = firstTaskAttempts.find(_.executorId == "executor0").get
    +    taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0))
    +    // we explicitly call the handleFailedTask method here to avoid adding a sleep in the test suite
    +    // Reason being - handleFailedTask is run by an executor service and there is a momentary delay
    +    // before it is launched and this fails the assertion check.
    +    tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, UnknownReason)
    +    when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask(
    +      "executor0", failedTask.index)).thenReturn(true)
    +
    +    // make an offer on the blacklisted executor.  We won't schedule anything, and set the abort
    +    // timer to kick in immediately
    +    assert(taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten.size === 0)
    +    // Wait for the abort timer to kick in. Without sleep the test exits before the timer is
    +    // triggered.
    +    eventually(timeout(500.milliseconds)) {
    +      assert(tsm.isZombie)
    +    }
    +  }
    +
    +  test("SPARK-22148 try to acquire a new executor when task is unschedulable with 1 executor") {
    +    taskScheduler = setupSchedulerWithMockTaskSetBlacklist(
    +      config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "10")
    +
    +    // We have only 1 task remaining with 1 executor
    +    val taskSet = FakeTask.createTaskSet(numTasks = 1, stageAttemptId = 0)
    +    taskScheduler.submitTasks(taskSet)
    +    val tsm = stageToMockTaskSetManager(0)
    +
    +    // submit an offer with one executor
    +    val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten
    +
    +    // Fail the running task
    +    val failedTask = firstTaskAttempts.find(_.executorId == "executor0").get
    +    taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0))
    +    // we explicitly call the handleFailedTask method here to avoid adding a sleep in the test suite
    +    // Reason being - handleFailedTask is run by an executor service and there is a momentary delay
    +    // before it is launched and this fails the assertion check.
    +    tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, UnknownReason)
    +    when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask(
    +      "executor0", failedTask.index)).thenReturn(true)
    +
    +    // make an offer on the blacklisted executor.  We won't schedule anything, and set the abort
    +    // timer to expire if no new executors could be acquired. We kill the existing idle blacklisted
    +    // executor and try to acquire a new one.
    +    assert(taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten.size === 0)
    +    assert(!tsm.isZombie)
    +
    +    // Offer a new executor which should be accepted
    +    assert(taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor1", "host0", 1)
    +    )).flatten.size === 1)
    +
    +    assert(!tsm.isZombie)
    +  }
    +
    +  // This is to test a scenario where we have two taskSets completely blacklisted and on acquiring
    +  // a new executor we don't want the abort timer for the second taskSet to expire and abort the job
    +  test("SPARK-22148 abort timer should clear unschedulableTaskSetToExpiryTime for all TaskSets") {
    +    taskScheduler = setupSchedulerWithMockTaskSetBlacklist(
    +      config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "10")
    +
    +    // We have 2 taskSets with 1 task remaining in each with 1 executor completely blacklisted
    +    val taskSet1 = FakeTask.createTaskSet(numTasks = 1, stageId = 0, stageAttemptId = 0)
    +    taskScheduler.submitTasks(taskSet1)
    +    val taskSet2 = FakeTask.createTaskSet(numTasks = 1, stageId = 1, stageAttemptId = 0)
    +    taskScheduler.submitTasks(taskSet2)
    +    val tsm = stageToMockTaskSetManager(0)
    +
    +    // submit an offer with one executor
    +    val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten
    +
    +    // Fail the running task
    +    val failedTask = firstTaskAttempts.find(_.executorId == "executor0").get
    +    taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0))
    +    when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask(
    +      "executor0", failedTask.index)).thenReturn(true)
    +
    +    // make an offer. We will schedule the task from the second taskSet. Since a task was scheduled
    +    // we do not kick off the abort timer for taskSet1
    +    val secondTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten
    --- End diff --
    
    I think after all of these calls to `resourceOffers`, you should add an assert on `unschedulableTaskSetToExpiryTime`, it would make it easier to understand what was happening and figure out quickly when things go wrong.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #98325 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98325/testReport)** for PR 22288 at commit [`676be55`](https://github.com/apache/spark/commit/676be551f84bc1d5304a128be349fe3b017f6925).


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #97625 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97625/testReport)** for PR 22288 at commit [`b2d0d40`](https://github.com/apache/spark/commit/b2d0d40771534291bdd5a1e3ebfc2c0c227c5956).
     * This patch **fails PySpark pip packaging tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #97622 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97622/testReport)** for PR 22288 at commit [`b2d0d40`](https://github.com/apache/spark/commit/b2d0d40771534291bdd5a1e3ebfc2c0c227c5956).
     * This patch **fails MiMa tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #98325 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98325/testReport)** for PR 22288 at commit [`676be55`](https://github.com/apache/spark/commit/676be551f84bc1d5304a128be349fe3b017f6925).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #97168 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97168/testReport)** for PR 22288 at commit [`2ac135b`](https://github.com/apache/spark/commit/2ac135b2def99cb8de9c525dbaacc1c32f3a680a).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r226464912
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +420,65 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case taskIndex: Some[Int] => // Returns the taskIndex which was unschedulable
    --- End diff --
    
    `case Some(taskIndex) =>`
    
    avoids the `taskIndex.get` later


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r228642238
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +420,54 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case Some(taskIndex) => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we try to find an existing idle blacklisted
    +              // executor. If we cannot find one, we abort immediately. Else we kill the idle
    +              // executor and kick off an abortTimer which if it doesn't schedule a task within the
    +              // the timeout will abort the taskSet if we were unable to schedule any task from the
    +              // taskSet.
    +              // Note 1: We keep track of schedulability on a per taskSet basis rather than on a per
    +              // task basis.
    +              // Note 2: The taskSet can still be aborted when there are more than one idle
    +              // blacklisted executors and dynamic allocation is on. This can happen when a killed
    +              // idle executor isn't replaced in time by ExecutorAllocationManager as it relies on
    +              // pending tasks and doesn't kill executors on idle timeouts, resulting in the abort
    +              // timer to expire and abort the taskSet.
    +              executorIdToRunningTaskIds.find(x => !isExecutorBusy(x._1)) match {
    +                case Some ((executorId, _)) =>
    +                  if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) {
    +                    blacklistTrackerOpt.foreach(blt => blt.killBlacklistedIdleExecutor(executorId))
    +
    +                    val timeout = conf.get(config.UNSCHEDULABLE_TASKSET_TIMEOUT) * 1000
    +                    unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis() + timeout
    +                    logInfo(s"Waiting for $timeout ms for completely "
    +                      + s"blacklisted task to be schedulable again before aborting $taskSet.")
    +                    abortTimer.schedule(
    +                      createUnschedulableTaskSetAbortTimer(taskSet, taskIndex), timeout)
    +                  }
    +                case _ => // Abort Immediately
    +                  logInfo("Cannot schedule any task because of complete blacklisting. No idle" +
    +                    s" executors can be found to kill. Aborting $taskSet." )
    +                  taskSet.abortSinceCompletelyBlacklisted(taskIndex)
    +              }
    +            case _ => // Do nothing if no tasks completely blacklisted.
    +          }
    +        } else {
    +          // We want to defer killing any taskSets as long as we have a non blacklisted executor
    +          // which can be used to schedule a task from any active taskSets. This ensures that the
    +          // job can make progress and if we encounter a flawed taskSet it will eventually either
    +          // fail or abort due to being completely blacklisted.
    --- End diff --
    
    @tgravescs since we've been back and forth on the discussion of the cases here, just want to make sure you're aware of the possibility for waiting indefinitely here. 


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r224833138
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -146,21 +146,31 @@ private[scheduler] class BlacklistTracker (
         nextExpiryTime = math.min(execMinExpiry, nodeMinExpiry)
       }
     
    +  private def killExecutor(exec: String, msg: String): Unit = {
    +    allocationClient match {
    +      case Some(a) =>
    +        logInfo(msg)
    +        a.killExecutors(Seq(exec), adjustTargetNumExecutors = false, countFailures = false,
    +          force = true)
    +      case None =>
    +        logInfo(s"Not attempting to kill blacklisted executor id $exec " +
    +          s"since allocation client is not defined.")
    +    }
    +  }
    +
       private def killBlacklistedExecutor(exec: String): Unit = {
         if (conf.get(config.BLACKLIST_KILL_ENABLED)) {
    -      allocationClient match {
    -        case Some(a) =>
    -          logInfo(s"Killing blacklisted executor id $exec " +
    -            s"since ${config.BLACKLIST_KILL_ENABLED.key} is set.")
    -          a.killExecutors(Seq(exec), adjustTargetNumExecutors = false, countFailures = false,
    -            force = true)
    -        case None =>
    -          logWarning(s"Not attempting to kill blacklisted executor id $exec " +
    -            s"since allocation client is not defined.")
    -      }
    +      killExecutor(exec,
    +        s"Killing blacklisted executor id $exec since ${config.BLACKLIST_KILL_ENABLED.key} is set.")
         }
       }
     
    +  private[scheduler] def killBlacklistedIdleExecutor(exec: String): Unit = {
    +    killExecutor(exec,
    --- End diff --
    
    Should this code be guarded by `if (conf.get(config.BLACKLIST_KILL_ENABLED))`? As the other `killBlacklistedExecutor` function?


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    >  I'm quite worry about this killing behaviour. I thik we should kill a executor iff it is idle.
    
    yes, you have a good point.  So the two extremes we need to consider are:
    
    1) One straggler task which happens to get stuck on a bad executor.  The task will fail on the executor, then when the executor is blacklisted, you've got nowhere else to run the task because DA has released the other executors.
    
    2) A flawed taskset, where the tasks will fail no matter what.
    
    In (2), we should think about other jobs running concurrently.  (You could have concurrent jobs with (1), but if there is another job you probably have other executors up, so its less likely.)
    
    It would be bad for us to kill things in scenario (2), where one bad taskset leads to us killing executors for other jobs.  But if we wait indefinitely for an idle executor to kill, then that taskset may wait indefinitely, which is also bad.
    
    > Maybe, we can add onTaskCompletelyBlacklisted() method in DA manager's Listener and pass a e.g. TaskCompletelyBlacklistedEvent to it. Thus, DA manger will allocate new executor for us.
    
    maybe this would help, lemme think about it ... I'd rather avoid adding this to the Listener api just for this as it should be an entirely internal detail, but maybe that is all we can do.  I guess this would let you bump up the request, as long as you're lower than the max executors, so it would solve the case when there is only one executor.  But in case 2, you'd probably end up requesting a whole bunch more executors very briefly, until there are enough failures on one specific task.  or maybe we can ensure that even if there are a huge number of unschedulable tasks, we only ever request one extra executor?
    
    > with static allocation
    > Set spark.scheduler.unschedulableTaskSetTimeout for a TaskSet. If a task blacklisted completely,
    kill some executors iff they're idle (Maybe, taking executors' allocation time into acount here, we should increase timeout upperbound for a little for this TaskSet.). Then, waiting until to be scheduled or timeout&abort.
    
    sorry I don't think I understand this part.  Is this the same as the current pr, but just killing only if idle?


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    (I'm on a outside trip these days, so I have to use my mobile phone to type these words. Sorry for the format.)
    
    > Is this the same as the current pr, but just killing only if idle?
    
    Yes, simillar. This avoids a TaskSet to wait to be scheduled indefinitely. So, in case 2, if we do not find a idle executor before timeout, the TaskSet would abort, rather than hang.
    
    > But in case 2, you'd probably end up requesting a whole bunch more executors very briefly, until there are enough failures on one specific task. or maybe we can ensure that even if there are a huge number of unschedulable tasks, we only ever request one extra executor?
    
    I'm not sure I have understand this part totally. But I realized a fact that, by now, our DA' strategy is basically based on tasks' status, e.g. pending, specatulative. However, a executor whether to be blacklisted depends on a success TaskSet' status (IIRC). So this fact may introduce level mismatch when we want to introduce DA in TaskScheduleImpl. (hope I understood your main thought)
    



---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r222816658
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +421,63 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    -        }
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case taskIndex: Some[Int] => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we try to find an existing idle blacklisted
    +              // executor. If we cannot find one, we abort immediately. Else we kill the idle
    +              // executor and kick off an abortTimer which after waiting will abort the taskSet if
    +              // we were unable to schedule any task from the taskSet.
    +              // Note 1: We keep a track of schedulability on a per taskSet basis rather than on a
    +              // per task basis.
    +              // Note 2: The taskSet can still be aborted when there are more than one idle
    +              // blacklisted executors and dynamic allocation is on. This is because we rely on the
    +              // ExecutorAllocationManager to acquire a new executor based on the pending tasks and
    +              // it won't release any blacklisted executors which idle timeout after we kill an
    +              // executor to acquire a new one, resulting in the abort timer to expire and abort the
    +              // taskSet.
    +              executorIdToRunningTaskIds.find(x => !isExecutorBusy(x._1)) match {
    +                case Some (x) =>
    +                  val executorId = x._1
    +                  if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) {
    +                    blacklistTrackerOpt.foreach(blt => blt.killBlacklistedIdleExecutor(executorId))
    +
    +                    unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis()
    +                    val timeout = conf.get(config.UNSCHEDULABLE_TASKSET_TIMEOUT) * 1000
    +                    logInfo(s"Waiting for $timeout ms for completely "
    +                      + s"blacklisted task to be schedulable again before aborting $taskSet.")
    +                    abortTimer.schedule(new TimerTask() {
    +                      override def run() {
    +                        if (unschedulableTaskSetToExpiryTime.contains(taskSet) &&
    +                          (unschedulableTaskSetToExpiryTime(taskSet) + timeout)
    +                            <= clock.getTimeMillis()
    +                        ) {
    +                          logInfo("Cannot schedule any task because of complete blacklisting. " +
    +                            s"Wait time for scheduling expired. Aborting $taskSet.")
    +                          taskSet.abortSinceCompletelyBlacklisted(taskIndex.get)
    +                        } else {
    +                          this.cancel()
    +                        }
    +                      }
    +                    }, timeout)
    +                  }
    +                case _ => // Abort Immediately
    +                  logInfo("Cannot schedule any task because of complete blacklisting. No idle" +
    +                  s" executors could be found. Aborting $taskSet." )
    +                  taskSet.abortSinceCompletelyBlacklisted(taskIndex.get)
    +              }
    +            case _ => // Do nothing.
    --- End diff --
    
    perhaps expand to say do nothing if no tasks completely blacklisted.  It looks like the indentation is off here too but it might just be because of the diff and comments


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    @squito I have made the changes and updated the description. 


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    Build finished. Test PASSed.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    You mentioned in the description that you did some manual testing -- since this has been through some changes since the initial versions, can you do that again?  please be sure to run some manual tests with (a) flawed jobs on a small cluster, so it really should abort and (b) OK jobs but with a failed straggler when only one executor is still active, which should kill the executor and get a new one.  If you've already done that on relatively recent revision, that's fine too.


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/4101/
    Test PASSed.


---

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


[GitHub] spark pull request #22288: [SPARK-22148][Scheduler] Acquire new executors to...

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

    https://github.com/apache/spark/pull/22288#discussion_r216726755
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala ---
    @@ -623,8 +623,9 @@ private[spark] class TaskSetManager(
        *
        * It is possible that this taskset has become impossible to schedule *anywhere* due to the
        * blacklist.  The most common scenario would be if there are fewer executors than
    -   * spark.task.maxFailures. We need to detect this so we can fail the task set, otherwise the job
    -   * will hang.
    +   * spark.task.maxFailures. We need to detect this so we can avoid the job from being hung.
    +   * If dynamic allocation is enabled we try to acquire new executor/s by killing the existing one.
    +   * In case of static allocation we abort the taskSet immediately to fail the job.
    --- End diff --
    
    why do you want something different with static allocation?  If you kill an executor, static allocation will also request a replacement.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #97542 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97542/testReport)** for PR 22288 at commit [`d2af73d`](https://github.com/apache/spark/commit/d2af73d4bd10676169f4fddc9dd262c97c0a9967).


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #96443 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/96443/testReport)** for PR 22288 at commit [`4c88168`](https://github.com/apache/spark/commit/4c881680fdde32244030b54b44125ac217dacb0d).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark pull request #22288: [SPARK-22148][Scheduler] Acquire new executors to...

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

    https://github.com/apache/spark/pull/22288#discussion_r216788096
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -414,9 +425,48 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    -        }
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case taskIndex: Some[Int] => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we kill an existing blacklisted executor/s and
    +              // kick off an abortTimer which after waiting will abort the taskSet if we were
    +              // unable to schedule any task from the taskSet.
    +              // Note: We keep a track of schedulability on a per taskSet basis rather than on a
    +              // per task basis.
    +              val executor = hostToExecutors.valuesIterator.next().iterator.next()
    +              logDebug("Killing executor because of task unschedulability: " + executor)
    --- End diff --
    
    noted.


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r226475523
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +419,61 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case taskIndex: Some[Int] => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we try to find an existing idle blacklisted
    +              // executor. If we cannot find one, we abort immediately. Else we kill the idle
    +              // executor and kick off an abortTimer which if it doesn't schedule a task within the
    +              // the timeout will abort the taskSet if we were unable to schedule any task from the
    +              // taskSet.
    +              // Note 1: We keep track of schedulability on a per taskSet basis rather than on a per
    +              // task basis.
    +              // Note 2: The taskSet can still be aborted when there are more than one idle
    +              // blacklisted executors and dynamic allocation is on. This can happen when a killed
    +              // idle executor isn't replaced in time by ExecutorAllocationManager as it relies on
    +              // pending tasks and doesn't kill executors on idle timeouts, resulting in the abort
    +              // timer to expire and abort the taskSet.
    +              executorIdToRunningTaskIds.find(x => !isExecutorBusy(x._1)) match {
    +                case Some (x) =>
    +                  val executorId = x._1
    +                  if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) {
    +                    blacklistTrackerOpt.foreach(blt => blt.killBlacklistedIdleExecutor(executorId))
    +
    +                    unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis()
    +                    val timeout = conf.get(config.UNSCHEDULABLE_TASKSET_TIMEOUT) * 1000
    +                    logInfo(s"Waiting for $timeout ms for completely "
    +                      + s"blacklisted task to be schedulable again before aborting $taskSet.")
    +                    abortTimer.schedule(new TimerTask() {
    +                      override def run() {
    +                        if (unschedulableTaskSetToExpiryTime.contains(taskSet) &&
    +                          (unschedulableTaskSetToExpiryTime(taskSet) + timeout)
    +                            <= clock.getTimeMillis()
    --- End diff --
    
    how about factoring out this whole run() method to a helper?  just to get rid of some indentation for better formatting


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r226477452
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +420,65 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case taskIndex: Some[Int] => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we try to find an existing idle blacklisted
    +              // executor. If we cannot find one, we abort immediately. Else we kill the idle
    +              // executor and kick off an abortTimer which if it doesn't schedule a task within the
    +              // the timeout will abort the taskSet if we were unable to schedule any task from the
    +              // taskSet.
    +              // Note 1: We keep track of schedulability on a per taskSet basis rather than on a per
    +              // task basis.
    +              // Note 2: The taskSet can still be aborted when there are more than one idle
    +              // blacklisted executors and dynamic allocation is on. This can happen when a killed
    +              // idle executor isn't replaced in time by ExecutorAllocationManager as it relies on
    +              // pending tasks and doesn't kill executors on idle timeouts, resulting in the abort
    +              // timer to expire and abort the taskSet.
    +              executorIdToRunningTaskIds.find(x => !isExecutorBusy(x._1)) match {
    +                case Some (x) =>
    +                  val executorId = x._1
    +                  if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) {
    +                    blacklistTrackerOpt.foreach(blt => blt.killBlacklistedIdleExecutor(executorId))
    +
    +                    unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis()
    +                    val timeout = conf.get(config.UNSCHEDULABLE_TASKSET_TIMEOUT) * 1000
    +                    logInfo(s"Waiting for $timeout ms for completely "
    +                      + s"blacklisted task to be schedulable again before aborting $taskSet.")
    +                    abortTimer.schedule(new TimerTask() {
    +                      override def run() {
    +                        if (unschedulableTaskSetToExpiryTime.contains(taskSet) &&
    +                          (unschedulableTaskSetToExpiryTime(taskSet) + timeout)
    +                            <= clock.getTimeMillis()
    +                        ) {
    +                          logInfo("Cannot schedule any task because of complete blacklisting. " +
    +                            s"Wait time for scheduling expired. Aborting $taskSet.")
    +                          taskSet.abortSinceCompletelyBlacklisted(taskIndex.get)
    +                        } else {
    +                          this.cancel()
    +                        }
    +                      }
    +                    }, timeout)
    +                  }
    +                case _ => // Abort Immediately
    +                  logInfo("Cannot schedule any task because of complete blacklisting. No idle" +
    +                    s" executors can be found to kill. Aborting $taskSet." )
    +                  taskSet.abortSinceCompletelyBlacklisted(taskIndex.get)
    +              }
    +            case _ => // Do nothing if no tasks completely blacklisted.
    +          }
    +        } else {
    +          // If a task was scheduled, we clear the expiry time for all the taskSets. This ensures
    +          // that we have got atleast a non blacklisted executor and the job can progress. The
    +          // abort timer checks this entry to decide if we want to abort the taskSet.
    --- End diff --
    
    typo: at least
    
    Want to make sure I understand this part, and why you aren't only clearing the timer for the taskset you just scheduled a task for.  If you have multiple tasksets running simultaneously, one is making progress but the other is totally blacklisted, I guess you do not want to kill anything, because that would mess with the taskset that is working correctly?  Instead you'll just let the taskset which is totally blacklisted eventually fail from the timeout?  I guess that makes sense, because if one taskset is progressing, it means the failing taskset probably is probably flawed, not the executors.
    
    If that's right, would be good to include something along those lines in the comment  (personally I don't find a comment about how its related to the timer that useful, that's obvious from the code).


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/4134/
    Test PASSed.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/4097/
    Test PASSed.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    +1 


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r228788350
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +420,54 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case Some(taskIndex) => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we try to find an existing idle blacklisted
    +              // executor. If we cannot find one, we abort immediately. Else we kill the idle
    +              // executor and kick off an abortTimer which if it doesn't schedule a task within the
    +              // the timeout will abort the taskSet if we were unable to schedule any task from the
    +              // taskSet.
    +              // Note 1: We keep track of schedulability on a per taskSet basis rather than on a per
    +              // task basis.
    +              // Note 2: The taskSet can still be aborted when there are more than one idle
    +              // blacklisted executors and dynamic allocation is on. This can happen when a killed
    +              // idle executor isn't replaced in time by ExecutorAllocationManager as it relies on
    +              // pending tasks and doesn't kill executors on idle timeouts, resulting in the abort
    +              // timer to expire and abort the taskSet.
    +              executorIdToRunningTaskIds.find(x => !isExecutorBusy(x._1)) match {
    +                case Some ((executorId, _)) =>
    +                  if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) {
    +                    blacklistTrackerOpt.foreach(blt => blt.killBlacklistedIdleExecutor(executorId))
    +
    +                    val timeout = conf.get(config.UNSCHEDULABLE_TASKSET_TIMEOUT) * 1000
    +                    unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis() + timeout
    +                    logInfo(s"Waiting for $timeout ms for completely "
    +                      + s"blacklisted task to be schedulable again before aborting $taskSet.")
    +                    abortTimer.schedule(
    +                      createUnschedulableTaskSetAbortTimer(taskSet, taskIndex), timeout)
    +                  }
    +                case _ => // Abort Immediately
    +                  logInfo("Cannot schedule any task because of complete blacklisting. No idle" +
    +                    s" executors can be found to kill. Aborting $taskSet." )
    +                  taskSet.abortSinceCompletelyBlacklisted(taskIndex)
    +              }
    +            case _ => // Do nothing if no tasks completely blacklisted.
    +          }
    +        } else {
    +          // We want to defer killing any taskSets as long as we have a non blacklisted executor
    +          // which can be used to schedule a task from any active taskSets. This ensures that the
    +          // job can make progress and if we encounter a flawed taskSet it will eventually either
    +          // fail or abort due to being completely blacklisted.
    --- End diff --
    
    so its a bit worse than regular starvation from having competing tasksets, as in this case you might actually have resources available on your cluster, but you never ask for them, because the executor allocation manager thinks you have enough based on the number of pending tasks.
    
    In any case, I agree this is a stretch, and overall its an improvement, so I'm OK with it.


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r227060905
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -453,6 +504,25 @@ private[spark] class TaskSchedulerImpl(
         return tasks
       }
     
    +  private def createUnschedulableTaskSetAbortTimer(
    +      taskSet: TaskSetManager,
    +      taskIndex: Int,
    +      timeout: Long): TimerTask = {
    +    new TimerTask() {
    +      override def run() {
    +        if (unschedulableTaskSetToExpiryTime.contains(taskSet) &&
    +          unschedulableTaskSetToExpiryTime(taskSet) <= clock.getTimeMillis()
    +        ) {
    --- End diff --
    
    nit: I think the style is to double-indent the second line of the continuation, and close it at the end of that line
    
    ```scala
    if (unschedulableTaskSetToExpiryTime.contains(taskSet) &&
        unschedulableTaskSetToExpiryTime(taskSet) <= clock.getTimeMillis()) {
      logInfo
    ```


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #97168 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97168/testReport)** for PR 22288 at commit [`2ac135b`](https://github.com/apache/spark/commit/2ac135b2def99cb8de9c525dbaacc1c32f3a680a).


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/3836/
    Test PASSed.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    @dhruve  is this ready to review again?


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r227069285
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +420,55 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case Some(taskIndex) => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we try to find an existing idle blacklisted
    +              // executor. If we cannot find one, we abort immediately. Else we kill the idle
    +              // executor and kick off an abortTimer which if it doesn't schedule a task within the
    +              // the timeout will abort the taskSet if we were unable to schedule any task from the
    +              // taskSet.
    +              // Note 1: We keep track of schedulability on a per taskSet basis rather than on a per
    +              // task basis.
    +              // Note 2: The taskSet can still be aborted when there are more than one idle
    +              // blacklisted executors and dynamic allocation is on. This can happen when a killed
    +              // idle executor isn't replaced in time by ExecutorAllocationManager as it relies on
    +              // pending tasks and doesn't kill executors on idle timeouts, resulting in the abort
    +              // timer to expire and abort the taskSet.
    +              executorIdToRunningTaskIds.find(x => !isExecutorBusy(x._1)) match {
    --- End diff --
    
    `isExecutorBusy` is going to probe the same hashmap again, you could just do `executorIdToRunningTaskIds.find(_._2.isEmpty)`


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][Scheduler] Acquire new executors to avoid ...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #95729 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95729/testReport)** for PR 22288 at commit [`87c4e57`](https://github.com/apache/spark/commit/87c4e57bb966078c8a78eabc5a5e4b6f60c78f28).


---

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


[GitHub] spark pull request #22288: [SPARK-22148][Scheduler] Acquire new executors to...

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

    https://github.com/apache/spark/pull/22288#discussion_r216725731
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -414,9 +425,48 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    -        }
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case taskIndex: Some[Int] => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we kill an existing blacklisted executor/s and
    +              // kick off an abortTimer which after waiting will abort the taskSet if we were
    +              // unable to schedule any task from the taskSet.
    +              // Note: We keep a track of schedulability on a per taskSet basis rather than on a
    +              // per task basis.
    +              val executor = hostToExecutors.valuesIterator.next().iterator.next()
    +              logDebug("Killing executor because of task unschedulability: " + executor)
    +              blacklistTrackerOpt.foreach(blt => blt.killBlacklistedExecutor(executor))
    +
    +              if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) {
    +                  unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis()
    --- End diff --
    
    I'd include a logInfo here that spark can't schedule anything because of blacklisting, but its going to try to kill blacklisted executors and acquire new ones.  Also mention how long it will wait before giving up and the associated conf.


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r222813512
  
    --- Diff: core/src/main/scala/org/apache/spark/internal/config/package.scala ---
    @@ -597,6 +597,17 @@ package object config {
           .checkValue(v => v > 0, "The value should be a positive time value.")
           .createWithDefaultString("365d")
     
    +  // Threshold above which we abort the TaskSet if a task could not be scheduled because of complete
    +  // blacklisting.
    +  private[spark] val UNSCHEDULABLE_TASKSET_TIMEOUT =
    +    ConfigBuilder("spark.scheduler.unschedulableTaskSetTimeout")
    +      .doc("The timeout in seconds to wait before aborting a TaskSet to acquire a new executor " +
    --- End diff --
    
    add blacklist to the name of the config, since it really only applies to blacklisted executors


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r224107221
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +419,61 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case taskIndex: Some[Int] => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we try to find an existing idle blacklisted
    +              // executor. If we cannot find one, we abort immediately. Else we kill the idle
    +              // executor and kick off an abortTimer which if it doesn't schedule a task within the
    +              // the timeout will abort the taskSet if we were unable to schedule any task from the
    +              // taskSet.
    +              // Note 1: We keep track of schedulability on a per taskSet basis rather than on a per
    +              // task basis.
    +              // Note 2: The taskSet can still be aborted when there are more than one idle
    +              // blacklisted executors and dynamic allocation is on. This can happen when a killed
    +              // idle executor isn't replaced in time by ExecutorAllocationManager as it relies on
    +              // pending tasks and doesn't kill executors on idle timeouts, resulting in the abort
    +              // timer to expire and abort the taskSet.
    +              executorIdToRunningTaskIds.find(x => !isExecutorBusy(x._1)) match {
    +                case Some (x) =>
    +                  val executorId = x._1
    +                  if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) {
    +                    blacklistTrackerOpt.foreach(blt => blt.killBlacklistedIdleExecutor(executorId))
    +
    +                    unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis()
    +                    val timeout = conf.get(config.UNSCHEDULABLE_TASKSET_TIMEOUT) * 1000
    +                    logInfo(s"Waiting for $timeout ms for completely "
    +                      + s"blacklisted task to be schedulable again before aborting $taskSet.")
    +                    abortTimer.schedule(new TimerTask() {
    +                      override def run() {
    +                        if (unschedulableTaskSetToExpiryTime.contains(taskSet) &&
    +                          (unschedulableTaskSetToExpiryTime(taskSet) + timeout)
    +                            <= clock.getTimeMillis()
    --- End diff --
    
    can you put this on previous line, looks like it should fit?


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    retest this please


---

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


[GitHub] spark issue #22288: [SPARK-22148][Scheduler] Acquire new executors to avoid ...

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

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


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r227000658
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +420,54 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case Some(taskIndex) => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we try to find an existing idle blacklisted
    +              // executor. If we cannot find one, we abort immediately. Else we kill the idle
    +              // executor and kick off an abortTimer which if it doesn't schedule a task within the
    +              // the timeout will abort the taskSet if we were unable to schedule any task from the
    +              // taskSet.
    +              // Note 1: We keep track of schedulability on a per taskSet basis rather than on a per
    +              // task basis.
    +              // Note 2: The taskSet can still be aborted when there are more than one idle
    +              // blacklisted executors and dynamic allocation is on. This can happen when a killed
    +              // idle executor isn't replaced in time by ExecutorAllocationManager as it relies on
    +              // pending tasks and doesn't kill executors on idle timeouts, resulting in the abort
    +              // timer to expire and abort the taskSet.
    +              executorIdToRunningTaskIds.find(x => !isExecutorBusy(x._1)) match {
    +                case Some (x) =>
    +                  val executorId = x._1
    +                  if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) {
    +                    blacklistTrackerOpt.foreach(blt => blt.killBlacklistedIdleExecutor(executorId))
    +
    +                    unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis()
    --- End diff --
    
     just put in the actual expiry time here rather then the current time


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r227082382
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +420,55 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case Some(taskIndex) => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we try to find an existing idle blacklisted
    +              // executor. If we cannot find one, we abort immediately. Else we kill the idle
    +              // executor and kick off an abortTimer which if it doesn't schedule a task within the
    +              // the timeout will abort the taskSet if we were unable to schedule any task from the
    +              // taskSet.
    +              // Note 1: We keep track of schedulability on a per taskSet basis rather than on a per
    +              // task basis.
    +              // Note 2: The taskSet can still be aborted when there are more than one idle
    +              // blacklisted executors and dynamic allocation is on. This can happen when a killed
    +              // idle executor isn't replaced in time by ExecutorAllocationManager as it relies on
    +              // pending tasks and doesn't kill executors on idle timeouts, resulting in the abort
    +              // timer to expire and abort the taskSet.
    +              executorIdToRunningTaskIds.find(x => !isExecutorBusy(x._1)) match {
    --- End diff --
    
    I was preferring the code to be more readable. As this isn't a frequently running scenario, may be we could keep it. Thoughts?


---

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


[GitHub] spark pull request #22288: [SPARK-22148][Scheduler] Acquire new executors to...

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

    https://github.com/apache/spark/pull/22288#discussion_r216788079
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -414,9 +425,48 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    -        }
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case taskIndex: Some[Int] => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we kill an existing blacklisted executor/s and
    +              // kick off an abortTimer which after waiting will abort the taskSet if we were
    +              // unable to schedule any task from the taskSet.
    +              // Note: We keep a track of schedulability on a per taskSet basis rather than on a
    +              // per task basis.
    +              val executor = hostToExecutors.valuesIterator.next().iterator.next()
    +              logDebug("Killing executor because of task unschedulability: " + executor)
    +              blacklistTrackerOpt.foreach(blt => blt.killBlacklistedExecutor(executor))
    +
    +              if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) {
    +                  unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis()
    --- End diff --
    
    noted.


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r224892495
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala ---
    @@ -146,21 +146,31 @@ private[scheduler] class BlacklistTracker (
         nextExpiryTime = math.min(execMinExpiry, nodeMinExpiry)
       }
     
    +  private def killExecutor(exec: String, msg: String): Unit = {
    +    allocationClient match {
    +      case Some(a) =>
    +        logInfo(msg)
    +        a.killExecutors(Seq(exec), adjustTargetNumExecutors = false, countFailures = false,
    +          force = true)
    +      case None =>
    +        logInfo(s"Not attempting to kill blacklisted executor id $exec " +
    +          s"since allocation client is not defined.")
    +    }
    +  }
    +
       private def killBlacklistedExecutor(exec: String): Unit = {
         if (conf.get(config.BLACKLIST_KILL_ENABLED)) {
    -      allocationClient match {
    -        case Some(a) =>
    -          logInfo(s"Killing blacklisted executor id $exec " +
    -            s"since ${config.BLACKLIST_KILL_ENABLED.key} is set.")
    -          a.killExecutors(Seq(exec), adjustTargetNumExecutors = false, countFailures = false,
    -            force = true)
    -        case None =>
    -          logWarning(s"Not attempting to kill blacklisted executor id $exec " +
    -            s"since allocation client is not defined.")
    -      }
    +      killExecutor(exec,
    +        s"Killing blacklisted executor id $exec since ${config.BLACKLIST_KILL_ENABLED.key} is set.")
         }
       }
     
    +  private[scheduler] def killBlacklistedIdleExecutor(exec: String): Unit = {
    +    killExecutor(exec,
    --- End diff --
    
    It doesn't make sense to have a flag for it. Because if you have it `off`, your job would always fail when you encounter all the executors are blacklisted and you can't schedule any task.


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r227069559
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +420,55 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case Some(taskIndex) => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we try to find an existing idle blacklisted
    +              // executor. If we cannot find one, we abort immediately. Else we kill the idle
    +              // executor and kick off an abortTimer which if it doesn't schedule a task within the
    +              // the timeout will abort the taskSet if we were unable to schedule any task from the
    +              // taskSet.
    +              // Note 1: We keep track of schedulability on a per taskSet basis rather than on a per
    +              // task basis.
    +              // Note 2: The taskSet can still be aborted when there are more than one idle
    +              // blacklisted executors and dynamic allocation is on. This can happen when a killed
    +              // idle executor isn't replaced in time by ExecutorAllocationManager as it relies on
    +              // pending tasks and doesn't kill executors on idle timeouts, resulting in the abort
    +              // timer to expire and abort the taskSet.
    +              executorIdToRunningTaskIds.find(x => !isExecutorBusy(x._1)) match {
    +                case Some (x) =>
    +                  val executorId = x._1
    --- End diff --
    
    just a suggestion, use more pattern matching:
    
    `case Some((executorId, _)) =>`


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/3856/
    Test PASSed.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/3830/
    Test PASSed.


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r228604322
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +420,54 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case Some(taskIndex) => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we try to find an existing idle blacklisted
    +              // executor. If we cannot find one, we abort immediately. Else we kill the idle
    +              // executor and kick off an abortTimer which if it doesn't schedule a task within the
    +              // the timeout will abort the taskSet if we were unable to schedule any task from the
    +              // taskSet.
    +              // Note 1: We keep track of schedulability on a per taskSet basis rather than on a per
    +              // task basis.
    +              // Note 2: The taskSet can still be aborted when there are more than one idle
    +              // blacklisted executors and dynamic allocation is on. This can happen when a killed
    +              // idle executor isn't replaced in time by ExecutorAllocationManager as it relies on
    +              // pending tasks and doesn't kill executors on idle timeouts, resulting in the abort
    +              // timer to expire and abort the taskSet.
    +              executorIdToRunningTaskIds.find(x => !isExecutorBusy(x._1)) match {
    +                case Some ((executorId, _)) =>
    +                  if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) {
    +                    blacklistTrackerOpt.foreach(blt => blt.killBlacklistedIdleExecutor(executorId))
    +
    +                    val timeout = conf.get(config.UNSCHEDULABLE_TASKSET_TIMEOUT) * 1000
    +                    unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis() + timeout
    +                    logInfo(s"Waiting for $timeout ms for completely "
    +                      + s"blacklisted task to be schedulable again before aborting $taskSet.")
    +                    abortTimer.schedule(
    +                      createUnschedulableTaskSetAbortTimer(taskSet, taskIndex), timeout)
    +                  }
    +                case _ => // Abort Immediately
    +                  logInfo("Cannot schedule any task because of complete blacklisting. No idle" +
    +                    s" executors can be found to kill. Aborting $taskSet." )
    +                  taskSet.abortSinceCompletelyBlacklisted(taskIndex)
    +              }
    +            case _ => // Do nothing if no tasks completely blacklisted.
    --- End diff --
    
    you can remove this case if instead above you do
    
    ``` scala
    taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors).foreach { taskIndex =>
    ```


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/4136/
    Test PASSed.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r222785940
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -117,6 +117,12 @@ private[spark] class TaskSchedulerImpl(
     
       protected val executorIdToHost = new HashMap[String, String]
     
    +  private val abortTimer = new Timer(true)
    +
    +  private val clock = new SystemClock
    +
    --- End diff --
    
    remove newline


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    @tgravescs I have addressed the review comments. 


---

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


[GitHub] spark pull request #22288: [SPARK-22148][Scheduler] Acquire new executors to...

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

    https://github.com/apache/spark/pull/22288#discussion_r216788016
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala ---
    @@ -623,8 +623,9 @@ private[spark] class TaskSetManager(
        *
        * It is possible that this taskset has become impossible to schedule *anywhere* due to the
        * blacklist.  The most common scenario would be if there are fewer executors than
    -   * spark.task.maxFailures. We need to detect this so we can fail the task set, otherwise the job
    -   * will hang.
    +   * spark.task.maxFailures. We need to detect this so we can avoid the job from being hung.
    +   * If dynamic allocation is enabled we try to acquire new executor/s by killing the existing one.
    +   * In case of static allocation we abort the taskSet immediately to fail the job.
    --- End diff --
    
    Yes. The change of removing a single executor takes care of static allocation as well. I will update the comments. 


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r227111412
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +420,55 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case Some(taskIndex) => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we try to find an existing idle blacklisted
    +              // executor. If we cannot find one, we abort immediately. Else we kill the idle
    --- End diff --
    
    I don't think this is true -- if there is no idle executor here, you abort the taskset immediately, you're not starting any timer, from this case lower down: `case _ => // Abort Immediately`.
    
    I think to do what you described, you would instead need to do something different in that case, like start the same abortTimer, and also set a flag `needToKillIdleExecutor` and then on every call to `resourceOffer`, check that flag and potentially find an executor to kill.  (However I haven't totally thought through that, not sure if it would really work.  again, I'm not saying this has to be addressed now, just thinking this through)


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r222810125
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +421,63 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    -        }
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case taskIndex: Some[Int] => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we try to find an existing idle blacklisted
    +              // executor. If we cannot find one, we abort immediately. Else we kill the idle
    +              // executor and kick off an abortTimer which after waiting will abort the taskSet if
    +              // we were unable to schedule any task from the taskSet.
    +              // Note 1: We keep a track of schedulability on a per taskSet basis rather than on a
    --- End diff --
    
    we keep track 


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r228668756
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +420,54 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case Some(taskIndex) => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we try to find an existing idle blacklisted
    +              // executor. If we cannot find one, we abort immediately. Else we kill the idle
    +              // executor and kick off an abortTimer which if it doesn't schedule a task within the
    +              // the timeout will abort the taskSet if we were unable to schedule any task from the
    +              // taskSet.
    +              // Note 1: We keep track of schedulability on a per taskSet basis rather than on a per
    +              // task basis.
    +              // Note 2: The taskSet can still be aborted when there are more than one idle
    +              // blacklisted executors and dynamic allocation is on. This can happen when a killed
    +              // idle executor isn't replaced in time by ExecutorAllocationManager as it relies on
    +              // pending tasks and doesn't kill executors on idle timeouts, resulting in the abort
    +              // timer to expire and abort the taskSet.
    +              executorIdToRunningTaskIds.find(x => !isExecutorBusy(x._1)) match {
    +                case Some ((executorId, _)) =>
    +                  if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) {
    +                    blacklistTrackerOpt.foreach(blt => blt.killBlacklistedIdleExecutor(executorId))
    +
    +                    val timeout = conf.get(config.UNSCHEDULABLE_TASKSET_TIMEOUT) * 1000
    +                    unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis() + timeout
    +                    logInfo(s"Waiting for $timeout ms for completely "
    +                      + s"blacklisted task to be schedulable again before aborting $taskSet.")
    +                    abortTimer.schedule(
    +                      createUnschedulableTaskSetAbortTimer(taskSet, taskIndex), timeout)
    +                  }
    +                case _ => // Abort Immediately
    +                  logInfo("Cannot schedule any task because of complete blacklisting. No idle" +
    +                    s" executors can be found to kill. Aborting $taskSet." )
    +                  taskSet.abortSinceCompletelyBlacklisted(taskIndex)
    +              }
    +            case _ => // Do nothing if no tasks completely blacklisted.
    +          }
    +        } else {
    +          // We want to defer killing any taskSets as long as we have a non blacklisted executor
    +          // which can be used to schedule a task from any active taskSets. This ensures that the
    +          // job can make progress and if we encounter a flawed taskSet it will eventually either
    +          // fail or abort due to being completely blacklisted.
    --- End diff --
    
    Thanks for pointing this out, but if I'm reading the discussion properly, I don't think you will actually wait indefinitely.  Eventually you will either abort immediately or you should fail due to max number of task failures.  Let me know if I'm missing something from the scenario.
    
    Lets say you have taskset1 that is blacklisted on all nodes (lets say we have 3). 3 cases can happen at this point:
    
    - taskset 2 hasn't started, so it tries to kill an executor and starts the timer.  
    - taskset 2 has started, if its running on all nodes then we abort immediately because no executors to kill to kill
    - taskset 2 has started but its not running on all blacklisted nodes, then we will kill an executor
    
    At this point lets say we didn't abort so we killed an executor.  Taskset 1 will get a chance to run on the new executor and either work or have a task failure.  If it has a task failure and it gets blacklisted, we go back into the case above.  But the # of task failures gets one closer.
    
    so it seems like eventually you would either abort immediately if there aren't any executors to kill or you would eventually fail with max number of task attempts.


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r228636880
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +420,54 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case Some(taskIndex) => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we try to find an existing idle blacklisted
    +              // executor. If we cannot find one, we abort immediately. Else we kill the idle
    +              // executor and kick off an abortTimer which if it doesn't schedule a task within the
    +              // the timeout will abort the taskSet if we were unable to schedule any task from the
    +              // taskSet.
    +              // Note 1: We keep track of schedulability on a per taskSet basis rather than on a per
    +              // task basis.
    +              // Note 2: The taskSet can still be aborted when there are more than one idle
    +              // blacklisted executors and dynamic allocation is on. This can happen when a killed
    +              // idle executor isn't replaced in time by ExecutorAllocationManager as it relies on
    +              // pending tasks and doesn't kill executors on idle timeouts, resulting in the abort
    +              // timer to expire and abort the taskSet.
    +              executorIdToRunningTaskIds.find(x => !isExecutorBusy(x._1)) match {
    +                case Some ((executorId, _)) =>
    +                  if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) {
    +                    blacklistTrackerOpt.foreach(blt => blt.killBlacklistedIdleExecutor(executorId))
    +
    +                    val timeout = conf.get(config.UNSCHEDULABLE_TASKSET_TIMEOUT) * 1000
    +                    unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis() + timeout
    +                    logInfo(s"Waiting for $timeout ms for completely "
    +                      + s"blacklisted task to be schedulable again before aborting $taskSet.")
    +                    abortTimer.schedule(
    +                      createUnschedulableTaskSetAbortTimer(taskSet, taskIndex), timeout)
    +                  }
    +                case _ => // Abort Immediately
    +                  logInfo("Cannot schedule any task because of complete blacklisting. No idle" +
    +                    s" executors can be found to kill. Aborting $taskSet." )
    +                  taskSet.abortSinceCompletelyBlacklisted(taskIndex)
    +              }
    +            case _ => // Do nothing if no tasks completely blacklisted.
    --- End diff --
    
    I have seen this style earlier in the code base. Is this a norm (just curious)? I read a few scenarios where this would be better. However, personally every time I read a foreach, its instinctive to think the entity on which its being invoked as an iterable rather than an option, so it feels a bit odd.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r222811527
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +421,63 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    -        }
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case taskIndex: Some[Int] => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we try to find an existing idle blacklisted
    +              // executor. If we cannot find one, we abort immediately. Else we kill the idle
    +              // executor and kick off an abortTimer which after waiting will abort the taskSet if
    +              // we were unable to schedule any task from the taskSet.
    +              // Note 1: We keep a track of schedulability on a per taskSet basis rather than on a
    +              // per task basis.
    +              // Note 2: The taskSet can still be aborted when there are more than one idle
    +              // blacklisted executors and dynamic allocation is on. This is because we rely on the
    +              // ExecutorAllocationManager to acquire a new executor based on the pending tasks and
    +              // it won't release any blacklisted executors which idle timeout after we kill an
    +              // executor to acquire a new one, resulting in the abort timer to expire and abort the
    +              // taskSet.
    +              executorIdToRunningTaskIds.find(x => !isExecutorBusy(x._1)) match {
    --- End diff --
    
    assume instead of x do (execid, tasks) then pass execid into isExecutorBusy


---

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


[GitHub] spark pull request #22288: [SPARK-22148][Scheduler] Acquire new executors to...

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

    https://github.com/apache/spark/pull/22288#discussion_r216723175
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -414,9 +425,48 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    -        }
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case taskIndex: Some[Int] => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we kill an existing blacklisted executor/s and
    +              // kick off an abortTimer which after waiting will abort the taskSet if we were
    +              // unable to schedule any task from the taskSet.
    +              // Note: We keep a track of schedulability on a per taskSet basis rather than on a
    +              // per task basis.
    +              val executor = hostToExecutors.valuesIterator.next().iterator.next()
    --- End diff --
    
    `hostToExecutors.head._2.head`
    
    just thinking "aloud" -- I guess taking an arbitrary executor here is OK, as we know there is some task that can't run on any executor.  But I wonder if we could have some priority here -- eg. I'd much rather kill an executor which has been blacklisted for an entire stage or the whole app, rather than one that was blacklisted for just some task.  Need to look into if there is an efficient way to keep that priority list, though.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/3828/
    Test PASSed.


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r227061367
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala ---
    @@ -503,6 +507,145 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
         verify(tsm).abort(anyString(), anyObject())
       }
     
    +  test("SPARK-22148 abort timer should kick in when task is completely blacklisted & no new " +
    +    "executor can be acquired") {
    --- End diff --
    
    nit: double indent this line


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    @squito I have tested it again with both scenarios and I was able to verify the expected behavior. For the cases that are not covered in the PR, i will mention them in the jira. 


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    test this please


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r222811152
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +421,63 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    -        }
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case taskIndex: Some[Int] => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we try to find an existing idle blacklisted
    +              // executor. If we cannot find one, we abort immediately. Else we kill the idle
    +              // executor and kick off an abortTimer which after waiting will abort the taskSet if
    +              // we were unable to schedule any task from the taskSet.
    +              // Note 1: We keep a track of schedulability on a per taskSet basis rather than on a
    +              // per task basis.
    +              // Note 2: The taskSet can still be aborted when there are more than one idle
    +              // blacklisted executors and dynamic allocation is on. This is because we rely on the
    --- End diff --
    
    I think the comment can be clarified/cleaned up a bit.
    
    I think the point here is that it can still be aborted if the executor we kill isn't replaced in time.    It doesn't explicitly idle timeout or kill other executors so the manager thinks it has enough.


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r223421728
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala ---
    @@ -503,6 +505,92 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
         verify(tsm).abort(anyString(), anyObject())
       }
     
    +  test("SPARK-22148 abort timer should kick in when task is completely blacklisted & no new " +
    +    "executor can be acquired") {
    +    // set the abort timer to fail immediately
    +    taskScheduler = setupSchedulerWithMockTaskSetBlacklist(
    +      config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "0")
    +
    +    // We have only 1 task remaining with 1 executor
    +    val taskSet = FakeTask.createTaskSet(numTasks = 1, stageAttemptId = 0)
    +    taskScheduler.submitTasks(taskSet)
    +    val tsm = stageToMockTaskSetManager(0)
    +
    +    // submit an offer with one executor
    +    val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten
    +
    +    // Fail the running task
    +    val failedTask = firstTaskAttempts.find(_.executorId == "executor0").get
    +    taskScheduler.statusUpdate(
    +      tid = failedTask.taskId,
    +      state = TaskState.FAILED,
    +      serializedData = ByteBuffer.allocate(0)
    +    )
    +    // Wait for the failed task to propagate.
    +    Thread.sleep(500)
    +    //    taskScheduler.handleFailedTask(tsm, failedTask.taskId, TaskState.FAILED, TaskResultLost)
    +    //    tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, TaskResultLost)
    +
    +    when(stageToMockTaskSetBlacklist(0).isExecutorBlacklistedForTask("executor0", failedTask.index))
    +      .thenReturn(true)
    +
    +    // make an offer on the blacklisted executor.  We won't schedule anything, and set the abort
    +    // timer to kick in immediately
    +    assert(taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten.size === 0)
    +    // Wait for the abort timer to kick in. Without sleep the test exits before the timer is
    +    // triggered.
    +    Thread.sleep(500)
    +    assert(tsm.isZombie)
    +  }
    +
    +  test("SPARK-22148 try to acquire a new executor when task is unschedulable with 1 executor") {
    +
    --- End diff --
    
    remove extra line


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r227080844
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala ---
    @@ -503,6 +507,145 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
         verify(tsm).abort(anyString(), anyObject())
       }
     
    +  test("SPARK-22148 abort timer should kick in when task is completely blacklisted & no new " +
    +    "executor can be acquired") {
    +    // set the abort timer to fail immediately
    +    taskScheduler = setupSchedulerWithMockTaskSetBlacklist(
    +      config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "0")
    +
    +    // We have only 1 task remaining with 1 executor
    +    val taskSet = FakeTask.createTaskSet(numTasks = 1, stageAttemptId = 0)
    +    taskScheduler.submitTasks(taskSet)
    +    val tsm = stageToMockTaskSetManager(0)
    +
    +    // submit an offer with one executor
    +    val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten
    +
    +    // Fail the running task
    +    val failedTask = firstTaskAttempts.find(_.executorId == "executor0").get
    +    taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0))
    +    // we explicitly call the handleFailedTask method here to avoid adding a sleep in the test suite
    +    // Reason being - handleFailedTask is run by an executor service and there is a momentary delay
    +    // before it is launched and this fails the assertion check.
    +    tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, UnknownReason)
    +    when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask(
    +      "executor0", failedTask.index)).thenReturn(true)
    +
    +    // make an offer on the blacklisted executor.  We won't schedule anything, and set the abort
    +    // timer to kick in immediately
    +    assert(taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten.size === 0)
    +    // Wait for the abort timer to kick in. Without sleep the test exits before the timer is
    +    // triggered.
    +    eventually(timeout(500.milliseconds)) {
    +      assert(tsm.isZombie)
    +    }
    +  }
    +
    +  test("SPARK-22148 try to acquire a new executor when task is unschedulable with 1 executor") {
    +    taskScheduler = setupSchedulerWithMockTaskSetBlacklist(
    +      config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "10")
    +
    +    // We have only 1 task remaining with 1 executor
    +    val taskSet = FakeTask.createTaskSet(numTasks = 1, stageAttemptId = 0)
    +    taskScheduler.submitTasks(taskSet)
    +    val tsm = stageToMockTaskSetManager(0)
    +
    +    // submit an offer with one executor
    +    val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten
    +
    +    // Fail the running task
    +    val failedTask = firstTaskAttempts.find(_.executorId == "executor0").get
    +    taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0))
    +    // we explicitly call the handleFailedTask method here to avoid adding a sleep in the test suite
    +    // Reason being - handleFailedTask is run by an executor service and there is a momentary delay
    +    // before it is launched and this fails the assertion check.
    +    tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, UnknownReason)
    +    when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask(
    +      "executor0", failedTask.index)).thenReturn(true)
    +
    +    // make an offer on the blacklisted executor.  We won't schedule anything, and set the abort
    +    // timer to expire if no new executors could be acquired. We kill the existing idle blacklisted
    +    // executor and try to acquire a new one.
    +    assert(taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten.size === 0)
    +    assert(!tsm.isZombie)
    +
    +    // Offer a new executor which should be accepted
    +    assert(taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor1", "host0", 1)
    +    )).flatten.size === 1)
    +
    +    assert(!tsm.isZombie)
    +  }
    +
    +  // This is to test a scenario where we have two taskSets completely blacklisted and on acquiring
    +  // a new executor we don't want the abort timer for the second taskSet to expire and abort the job
    +  test("SPARK-22148 abort timer should clear unschedulableTaskSetToExpiryTime for all TaskSets") {
    +    taskScheduler = setupSchedulerWithMockTaskSetBlacklist(
    +      config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "10")
    +
    +    // We have 2 taskSets with 1 task remaining in each with 1 executor completely blacklisted
    +    val taskSet1 = FakeTask.createTaskSet(numTasks = 1, stageId = 0, stageAttemptId = 0)
    +    taskScheduler.submitTasks(taskSet1)
    +    val taskSet2 = FakeTask.createTaskSet(numTasks = 1, stageId = 1, stageAttemptId = 0)
    +    taskScheduler.submitTasks(taskSet2)
    +    val tsm = stageToMockTaskSetManager(0)
    +
    +    // submit an offer with one executor
    +    val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten
    +
    +    // Fail the running task
    +    val failedTask = firstTaskAttempts.find(_.executorId == "executor0").get
    +    taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0))
    +    when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask(
    +      "executor0", failedTask.index)).thenReturn(true)
    +
    +    // make an offer. We will schedule the task from the second taskSet. Since a task was scheduled
    +    // we do not kick off the abort timer for taskSet1
    +    val secondTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten
    --- End diff --
    
    Okay.


---

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


[GitHub] spark pull request #22288: [SPARK-22148][Scheduler] Acquire new executors to...

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

    https://github.com/apache/spark/pull/22288#discussion_r215036162
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -414,9 +425,54 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    -        }
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case taskIndex: Some[Int] => // Returns the taskIndex which was unschedulable
    +              if (conf.getBoolean("spark.dynamicAllocation.enabled", false)) {
    +                // If the taskSet is unschedulable we kill the existing blacklisted executor/s and
    +                // kick off an abortTimer which after waiting will abort the taskSet if we were
    +                // unable to get new executors and couldn't schedule a task from the taskSet.
    +                // Note: We keep a track of schedulability on a per taskSet basis rather than on a
    +                // per task basis.
    +                if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) {
    +                  hostToExecutors.valuesIterator.foreach(executors => executors.foreach({
    +                    executor =>
    +                      logDebug("Killing executor because of task unschedulability: " + executor)
    +                      blacklistTrackerOpt.foreach(blt => blt.killBlacklistedExecutor(executor))
    --- End diff --
    
    - To refresh executors, you need to enable `spark.blacklist.killBlacklistedExecutors`.
    - I was thinking about it, killing all the executors is a little too harsh. Killing only a single executor  would help mitigate this, although this would also lead to failing the running tasks on that executor.


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r225998426
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +420,65 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case taskIndex: Some[Int] => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we try to find an existing idle blacklisted
    +              // executor. If we cannot find one, we abort immediately. Else we kill the idle
    +              // executor and kick off an abortTimer which if it doesn't schedule a task within the
    +              // the timeout will abort the taskSet if we were unable to schedule any task from the
    +              // taskSet.
    +              // Note 1: We keep track of schedulability on a per taskSet basis rather than on a per
    +              // task basis.
    +              // Note 2: The taskSet can still be aborted when there are more than one idle
    +              // blacklisted executors and dynamic allocation is on. This can happen when a killed
    +              // idle executor isn't replaced in time by ExecutorAllocationManager as it relies on
    +              // pending tasks and doesn't kill executors on idle timeouts, resulting in the abort
    +              // timer to expire and abort the taskSet.
    +              executorIdToRunningTaskIds.find(x => !isExecutorBusy(x._1)) match {
    +                case Some (x) =>
    +                  val executorId = x._1
    +                  if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) {
    +                    blacklistTrackerOpt.foreach(blt => blt.killBlacklistedIdleExecutor(executorId))
    +
    +                    unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis()
    +                    val timeout = conf.get(config.UNSCHEDULABLE_TASKSET_TIMEOUT) * 1000
    +                    logInfo(s"Waiting for $timeout ms for completely "
    +                      + s"blacklisted task to be schedulable again before aborting $taskSet.")
    +                    abortTimer.schedule(new TimerTask() {
    +                      override def run() {
    +                        if (unschedulableTaskSetToExpiryTime.contains(taskSet) &&
    +                          (unschedulableTaskSetToExpiryTime(taskSet) + timeout)
    +                            <= clock.getTimeMillis()
    +                        ) {
    +                          logInfo("Cannot schedule any task because of complete blacklisting. " +
    +                            s"Wait time for scheduling expired. Aborting $taskSet.")
    +                          taskSet.abortSinceCompletelyBlacklisted(taskIndex.get)
    +                        } else {
    +                          this.cancel()
    +                        }
    +                      }
    +                    }, timeout)
    +                  }
    +                case _ => // Abort Immediately
    +                  logInfo("Cannot schedule any task because of complete blacklisting. No idle" +
    +                  s" executors can be found to kill. Aborting $taskSet." )
    --- End diff --
    
    ```suggestion
                        s" executors can be found to kill. Aborting $taskSet." )
    ```


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #97879 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97879/testReport)** for PR 22288 at commit [`4a5ea82`](https://github.com/apache/spark/commit/4a5ea827263b8e789f03474d931105c707c6e778).


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    the failures seem to be unrelated. I wasn't able to reproduce them.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][Scheduler] Acquire new executors to avoid ...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r216874530
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -414,9 +425,48 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    -        }
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case taskIndex: Some[Int] => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we kill an existing blacklisted executor/s and
    +              // kick off an abortTimer which after waiting will abort the taskSet if we were
    +              // unable to schedule any task from the taskSet.
    +              // Note: We keep a track of schedulability on a per taskSet basis rather than on a
    +              // per task basis.
    +              val executor = hostToExecutors.valuesIterator.next().iterator.next()
    --- End diff --
    
    I'm wondering is it worth to kill someone executor which has some tasks running on it ? After all, a task blaklisted on all executors(currently allocated) can not be guaranteed to run on a new allocated executor.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/3574/
    Test PASSed.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    Thanks for the reviews and feedback @tgravescs , @squito !


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    @tgravescs I have fixed a nit and its good to be reviewed. @squito I have updated the comment, let me know if its okay.
    
    Thanks for the reviews.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    It applies to both DA and SA. I have updated the description.


---

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


[GitHub] spark pull request #22288: [SPARK-22148][Scheduler] Acquire new executors to...

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

    https://github.com/apache/spark/pull/22288#discussion_r216724373
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -414,9 +425,48 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    -        }
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case taskIndex: Some[Int] => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we kill an existing blacklisted executor/s and
    +              // kick off an abortTimer which after waiting will abort the taskSet if we were
    +              // unable to schedule any task from the taskSet.
    +              // Note: We keep a track of schedulability on a per taskSet basis rather than on a
    +              // per task basis.
    +              val executor = hostToExecutors.valuesIterator.next().iterator.next()
    +              logDebug("Killing executor because of task unschedulability: " + executor)
    --- End diff --
    
    I think this should probably be logInfo (unless there is something else similar at INFO level elsewhere)


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/4673/
    Test PASSed.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #97879 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97879/testReport)** for PR 22288 at commit [`4a5ea82`](https://github.com/apache/spark/commit/4a5ea827263b8e789f03474d931105c707c6e778).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][Scheduler] Acquire new executors to avoid ...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r225185389
  
    --- Diff: core/src/main/scala/org/apache/spark/internal/config/package.scala ---
    @@ -597,6 +597,16 @@ package object config {
           .checkValue(v => v > 0, "The value should be a positive time value.")
           .createWithDefaultString("365d")
     
    +  // Threshold above which we abort the TaskSet if a task could not be scheduled because of complete
    --- End diff --
    
    I don't think we need the extra comment, the doc section should be sufficient


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #96767 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/96767/testReport)** for PR 22288 at commit [`4c88168`](https://github.com/apache/spark/commit/4c881680fdde32244030b54b44125ac217dacb0d).


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r223421506
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala ---
    @@ -503,6 +505,92 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
         verify(tsm).abort(anyString(), anyObject())
       }
     
    +  test("SPARK-22148 abort timer should kick in when task is completely blacklisted & no new " +
    +    "executor can be acquired") {
    +    // set the abort timer to fail immediately
    +    taskScheduler = setupSchedulerWithMockTaskSetBlacklist(
    +      config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "0")
    +
    +    // We have only 1 task remaining with 1 executor
    +    val taskSet = FakeTask.createTaskSet(numTasks = 1, stageAttemptId = 0)
    +    taskScheduler.submitTasks(taskSet)
    +    val tsm = stageToMockTaskSetManager(0)
    +
    +    // submit an offer with one executor
    +    val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq(
    +      WorkerOffer("executor0", "host0", 1)
    +    )).flatten
    +
    +    // Fail the running task
    +    val failedTask = firstTaskAttempts.find(_.executorId == "executor0").get
    +    taskScheduler.statusUpdate(
    +      tid = failedTask.taskId,
    +      state = TaskState.FAILED,
    +      serializedData = ByteBuffer.allocate(0)
    +    )
    +    // Wait for the failed task to propagate.
    +    Thread.sleep(500)
    +    //    taskScheduler.handleFailedTask(tsm, failedTask.taskId, TaskState.FAILED, TaskResultLost)
    --- End diff --
    
    remove commented out code


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r224167756
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -415,9 +419,61 @@ private[spark] class TaskSchedulerImpl(
                 launchedAnyTask |= launchedTaskAtCurrentMaxLocality
               } while (launchedTaskAtCurrentMaxLocality)
             }
    +
             if (!launchedAnyTask) {
    -          taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
    +          taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors) match {
    +            case taskIndex: Some[Int] => // Returns the taskIndex which was unschedulable
    +
    +              // If the taskSet is unschedulable we try to find an existing idle blacklisted
    +              // executor. If we cannot find one, we abort immediately. Else we kill the idle
    +              // executor and kick off an abortTimer which if it doesn't schedule a task within the
    +              // the timeout will abort the taskSet if we were unable to schedule any task from the
    +              // taskSet.
    +              // Note 1: We keep track of schedulability on a per taskSet basis rather than on a per
    +              // task basis.
    +              // Note 2: The taskSet can still be aborted when there are more than one idle
    +              // blacklisted executors and dynamic allocation is on. This can happen when a killed
    +              // idle executor isn't replaced in time by ExecutorAllocationManager as it relies on
    +              // pending tasks and doesn't kill executors on idle timeouts, resulting in the abort
    +              // timer to expire and abort the taskSet.
    +              executorIdToRunningTaskIds.find(x => !isExecutorBusy(x._1)) match {
    +                case Some (x) =>
    +                  val executorId = x._1
    +                  if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) {
    +                    blacklistTrackerOpt.foreach(blt => blt.killBlacklistedIdleExecutor(executorId))
    +
    +                    unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis()
    +                    val timeout = conf.get(config.UNSCHEDULABLE_TASKSET_TIMEOUT) * 1000
    +                    logInfo(s"Waiting for $timeout ms for completely "
    +                      + s"blacklisted task to be schedulable again before aborting $taskSet.")
    +                    abortTimer.schedule(new TimerTask() {
    +                      override def run() {
    +                        if (unschedulableTaskSetToExpiryTime.contains(taskSet) &&
    +                          (unschedulableTaskSetToExpiryTime(taskSet) + timeout)
    +                            <= clock.getTimeMillis()
    --- End diff --
    
    it doesn't fit within the 100 char limit


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #97499 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97499/testReport)** for PR 22288 at commit [`69c156b`](https://github.com/apache/spark/commit/69c156b32cf573c7d3a75f63ab093022046ad856).


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r227070407
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala ---
    @@ -453,6 +504,25 @@ private[spark] class TaskSchedulerImpl(
         return tasks
       }
     
    +  private def createUnschedulableTaskSetAbortTimer(
    +      taskSet: TaskSetManager,
    +      taskIndex: Int,
    +      timeout: Long): TimerTask = {
    --- End diff --
    
    not using timeout here anymore


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    **[Test build #97176 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97176/testReport)** for PR 22288 at commit [`4ce7610`](https://github.com/apache/spark/commit/4ce7610522927ecabd0b8a75fc0a557938135b0d).


---

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


[GitHub] spark pull request #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new...

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

    https://github.com/apache/spark/pull/22288#discussion_r223451952
  
    --- Diff: core/src/main/scala/org/apache/spark/internal/config/package.scala ---
    @@ -597,6 +597,17 @@ package object config {
           .checkValue(v => v > 0, "The value should be a positive time value.")
           .createWithDefaultString("365d")
     
    +  // Threshold above which we abort the TaskSet if a task could not be scheduled because of complete
    +  // blacklisting.
    +  private[spark] val UNSCHEDULABLE_TASKSET_TIMEOUT =
    +    ConfigBuilder("spark.scheduler.unschedulableTaskSetTimeout")
    +      .doc("The timeout in seconds to wait before aborting a TaskSet to acquire a new executor " +
    --- End diff --
    
    also need to document in the .md file


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

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


---

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


[GitHub] spark issue #22288: [SPARK-22148][SPARK-15815][Scheduler] Acquire new execut...

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

    https://github.com/apache/spark/pull/22288
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/3349/
    Test PASSed.


---

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