You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by hthuynh2 <gi...@git.apache.org> on 2018/06/28 04:29:59 UTC

[GitHub] spark pull request #21653: [SPARK-13343] speculative tasks that didn't commi...

GitHub user hthuynh2 opened a pull request:

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

    [SPARK-13343] speculative tasks that didn't commit shouldn't be marked as success

    **Description**
    Currently Speculative tasks that didn't commit can show up as success of failures (depending on timing of commit). This is a bit confusing because that task didn't really succeed in the sense it didn't write anything.
    I think these tasks should be marked as KILLED or something that is more obvious to the user exactly what happened. it is happened to hit the timing where it got a commit denied exception then it shows up as failed and counts against your task failures. It shouldn't count against task failures since that failure really doesn't matter.
    MapReduce handles these situation so perhaps we can look there for a model.
    
    <img width="1420" alt="unknown" src="https://user-images.githubusercontent.com/15680678/42013170-99db48c2-7a61-11e8-8c7b-ef94c84e36ea.png">
    
    **How can this issue happen?**
    When both attempts of a task finish before the driver sends command to kill one of them, both of them send the status update FINISHED to the driver. The driver calls TaskSchedulerImpl to handle one successful task at a time. When it handles the first successful task, it sends the command to kill the other copy of the task, however, because that task is already finished, the executor will ignore the command. After finishing handling the first attempt, it processes the second one, although all actions on the result of this task are skipped, this copy of the task is still marked as SUCCESS. As a result, even though this issue does not affect the result of the job, it might cause confusing to user because both of them appear to be successful.
    
    **How does this PR fix the issue?**
    The simple way to fix this issue is that when taskSetManager handles successful task, it checks if any other attempt succeeded. If this is the case, it will call handleFailedTask with state==KILLED and reason==TaskKilled(“another attempt succeeded”) to handle this task as begin killed.
    
    **How was this patch tested?**
    I tested this manually by running applications, that caused the issue before, a few times, and observed that the issue does not happen again. Also, I added a unit test in TaskSetManagerSuite to test that if we call handleSuccessfulTask to handle status update for 2 copies of a task, only the one that is handled first will be mark as SUCCESS


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

    $ git pull https://github.com/hthuynh2/spark SPARK_13343

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

    https://github.com/apache/spark/pull/21653.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 #21653
    
----
commit 8f7d98177816e11659cf79a2b28f96bd4b7173d5
Author: Hieu Huynh <“h...@...>
Date:   2018-06-28T04:19:14Z

    Fixed issue and added unit test

----


---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

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


---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    @tgravescs I updated it. Can you please have a look at it when you have time. Thank you.


---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    Can one of the admins verify this patch?


---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    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 #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    @squito Thanks for the suggestions. I updated it. Could you please have a look at it to see if there is anything else I need to change? Thanks.


---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

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


---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    @tgravescs Can you please run the test again, thank you. 


---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    **[Test build #92426 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92426/testReport)** for PR 21653 at commit [`980a933`](https://github.com/apache/spark/commit/980a9333e4c9ed8752ec4c3f12d8f5407f84da5d).
     * 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 #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    #21729 has been merged @hthuynh2  can you update this one?


---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    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 #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

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


---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

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


---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    Can one of the admins verify this patch?


---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    lets review https://github.com/apache/spark/pull/21729 before this since its changing the type on killedByOtherAttempt


---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

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


---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    **[Test build #92690 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92690/testReport)** for PR 21653 at commit [`3c655f2`](https://github.com/apache/spark/commit/3c655f2b24e65602bb889b512a68fc598662a898).


---

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


[GitHub] spark pull request #21653: [SPARK-13343] speculative tasks that didn't commi...

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

    https://github.com/apache/spark/pull/21653#discussion_r204177708
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala ---
    @@ -723,6 +723,21 @@ private[spark] class TaskSetManager(
       def handleSuccessfulTask(tid: Long, result: DirectTaskResult[_]): Unit = {
         val info = taskInfos(tid)
         val index = info.index
    +    // Check if any other attempt succeeded before this and this attempt has not been handled
    +    if (successful(index) && killedByOtherAttempt.contains(tid)) {
    +      calculatedTasks -= 1
    +
    +      val resultSizeAcc = result.accumUpdates.find(a =>
    +        a.name == Some(InternalAccumulator.RESULT_SIZE))
    +      if (resultSizeAcc.isDefined) {
    +        totalResultSize -= resultSizeAcc.get.asInstanceOf[LongAccumulator].value
    --- End diff --
    
    the downside here is we already incremented and other tasks could have checked and failed before we decrement, but unless someone else has a better idea this is better then it is now. 


---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

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


---

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


[GitHub] spark pull request #21653: [SPARK-13343] speculative tasks that didn't commi...

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

    https://github.com/apache/spark/pull/21653#discussion_r199550367
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala ---
    @@ -1371,4 +1371,64 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
         val valueSer = SparkEnv.get.serializer.newInstance()
         new DirectTaskResult[Int](valueSer.serialize(id), accumUpdates)
       }
    +
    +  test("SPARK-13343 speculative tasks that didn't commit shouldn't be marked as success") {
    +    sc = new SparkContext("local", "test")
    +    sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2"))
    +    val taskSet = FakeTask.createTaskSet(4)
    +    // Set the speculation multiplier to be 0 so speculative tasks are launched immediately
    +    sc.conf.set("spark.speculation.multiplier", "0.0")
    +    sc.conf.set("spark.speculation", "true")
    +    val clock = new ManualClock()
    +    val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock)
    +    val accumUpdatesByTask: Array[Seq[AccumulatorV2[_, _]]] = taskSet.tasks.map { task =>
    +      task.metrics.internalAccums
    +    }
    +
    +    // Offer resources for 4 tasks to start
    +    for ((k, v) <- List(
    +      "exec1" -> "host1",
    +      "exec1" -> "host1",
    +      "exec2" -> "host2",
    +      "exec2" -> "host2")) {
    +      val taskOption = manager.resourceOffer(k, v, NO_PREF)
    +      assert(taskOption.isDefined)
    +      val task = taskOption.get
    +      assert(task.executorId === k)
    +    }
    +    assert(sched.startedTasks.toSet === Set(0, 1, 2, 3))
    +    clock.advance(1)
    +    // Complete the 3 tasks and leave 1 task in running
    +    for (id <- Set(0, 1, 2)) {
    +      manager.handleSuccessfulTask(id, createTaskResult(id, accumUpdatesByTask(id)))
    +      assert(sched.endedTasks(id) === Success)
    +    }
    +
    +    // checkSpeculatableTasks checks that the task runtime is greater than the threshold for
    +    // speculating. Since we use a threshold of 0 for speculation, tasks need to be running for
    +    // > 0ms, so advance the clock by 1ms here.
    +    clock.advance(1)
    +    assert(manager.checkSpeculatableTasks(0))
    +    assert(sched.speculativeTasks.toSet === Set(3))
    +
    +    // Offer resource to start the speculative attempt for the running task
    +    val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF)
    +    assert(taskOption5.isDefined)
    +    val task5 = taskOption5.get
    +    assert(task5.index === 3)
    +    assert(task5.taskId === 4)
    +    assert(task5.executorId === "exec1")
    +    assert(task5.attemptNumber === 1)
    +    sched.backend = mock(classOf[SchedulerBackend])
    +
    +    // Complete one attempt for the running task
    +    manager.handleSuccessfulTask(3, createTaskResult(3, accumUpdatesByTask(3)))
    +    // Verify that it kills other running attempt
    +    verify(sched.backend).killTask(4, "exec1", true, "another attempt succeeded")
    +    // Complete another attempt for the running task
    --- End diff --
    
    can you expand this comment to explain why you're doing this?  without looking at the bug, it's easy to think this part is wrong, but in fact its the most important part of your test.  eg:
    
    There is a race between the scheduler asking to kill the other task, and that task actually finishing.  We simulate what happens if the other task finishes before we kill it.


---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

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


---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    I updated it. Thanks.


---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    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 #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    **[Test build #92690 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92690/testReport)** for PR 21653 at commit [`3c655f2`](https://github.com/apache/spark/commit/3c655f2b24e65602bb889b512a68fc598662a898).
     * 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 #21653: [SPARK-13343] speculative tasks that didn't commi...

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

    https://github.com/apache/spark/pull/21653#discussion_r200805005
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala ---
    @@ -723,6 +723,13 @@ private[spark] class TaskSetManager(
       def handleSuccessfulTask(tid: Long, result: DirectTaskResult[_]): Unit = {
         val info = taskInfos(tid)
         val index = info.index
    +    // Check if any other attempt succeeded before this and this attempt has not been handled
    +    if (successful(index) && killedByOtherAttempt(index)) {
    --- End diff --
    
    For completeness, we will also need to 'undo' the changes in `enqueueSuccessfulTask` : to reverse the counters in `canFetchMoreResults`.
    
    
    (Orthogonal to this PR): Looking at use of `killedByOtherAttempt`, I see that there is a bug in `executorLost` w.r.t how it is updated - hopefully a fix for SPARK-24755 wont cause issues here.


---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    **[Test build #4222 has finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/4222/testReport)** for PR 21653 at commit [`b6585da`](https://github.com/apache/spark/commit/b6585da0f137d3d3675925368c4668c884de900c).
     * 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 #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    **[Test build #92426 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92426/testReport)** for PR 21653 at commit [`980a933`](https://github.com/apache/spark/commit/980a9333e4c9ed8752ec4c3f12d8f5407f84da5d).


---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    **[Test build #93447 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93447/testReport)** for PR 21653 at commit [`b6585da`](https://github.com/apache/spark/commit/b6585da0f137d3d3675925368c4668c884de900c).
     * 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 #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    @hthuynh2  can you fix the description "as success of failures" , this is just a copy of my typo in the jira.  Can you just change to be "as success"


---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    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 #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    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 #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    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 #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    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 #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

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


---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    I kicked off the test manually at https://spark-prs.appspot.com/users/hthuynh2.  I dunno why the test triggering via comments stops workign on some prs


---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    merged to master, thanks @hthuynh2 


---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    **[Test build #92688 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92688/testReport)** for PR 21653 at commit [`f66fcab`](https://github.com/apache/spark/commit/f66fcab04bcfdcbf0e7f1cac831205d7a2e7d666).
     * 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 #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    @jiangxb1987 yes, you are correct that it is actually ignored. I think it doesn't worth to add a new TaskState because we might need to add changes in many places but does not add much benefit. Instead, I think we can add some message to the kill message to differentiate it from task that is actually killed and to inform the user.  


---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    **[Test build #93474 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93474/testReport)** for PR 21653 at commit [`b6585da`](https://github.com/apache/spark/commit/b6585da0f137d3d3675925368c4668c884de900c).
     * This patch **fails due to an unknown error code, -9**.
     * 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 #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    **[Test build #93418 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93418/testReport)** for PR 21653 at commit [`b6585da`](https://github.com/apache/spark/commit/b6585da0f137d3d3675925368c4668c884de900c).
     * This patch **fails PySpark 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 #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

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


---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

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


---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    **[Test build #92425 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92425/testReport)** for PR 21653 at commit [`8f7d981`](https://github.com/apache/spark/commit/8f7d98177816e11659cf79a2b28f96bd4b7173d5).
     * 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 #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    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 #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    Can one of the admins verify this patch?


---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    IIUC this speculative task is not really killed right ? It is actually ignored. Does that worth it to add a new TaskState for this case ?


---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

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


---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    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 #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    ok to test


---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    cc @tgravescs 


---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    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 #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    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 #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    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 #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

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


---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    **[Test build #4222 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/4222/testReport)** for PR 21653 at commit [`b6585da`](https://github.com/apache/spark/commit/b6585da0f137d3d3675925368c4668c884de900c).


---

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


[GitHub] spark pull request #21653: [SPARK-13343] speculative tasks that didn't commi...

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

    https://github.com/apache/spark/pull/21653#discussion_r199550557
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala ---
    @@ -1371,4 +1371,64 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
         val valueSer = SparkEnv.get.serializer.newInstance()
         new DirectTaskResult[Int](valueSer.serialize(id), accumUpdates)
       }
    +
    +  test("SPARK-13343 speculative tasks that didn't commit shouldn't be marked as success") {
    +    sc = new SparkContext("local", "test")
    +    sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2"))
    +    val taskSet = FakeTask.createTaskSet(4)
    +    // Set the speculation multiplier to be 0 so speculative tasks are launched immediately
    +    sc.conf.set("spark.speculation.multiplier", "0.0")
    +    sc.conf.set("spark.speculation", "true")
    +    val clock = new ManualClock()
    +    val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock)
    +    val accumUpdatesByTask: Array[Seq[AccumulatorV2[_, _]]] = taskSet.tasks.map { task =>
    +      task.metrics.internalAccums
    +    }
    +
    +    // Offer resources for 4 tasks to start
    +    for ((k, v) <- List(
    +      "exec1" -> "host1",
    +      "exec1" -> "host1",
    +      "exec2" -> "host2",
    +      "exec2" -> "host2")) {
    +      val taskOption = manager.resourceOffer(k, v, NO_PREF)
    +      assert(taskOption.isDefined)
    +      val task = taskOption.get
    +      assert(task.executorId === k)
    +    }
    +    assert(sched.startedTasks.toSet === Set(0, 1, 2, 3))
    +    clock.advance(1)
    +    // Complete the 3 tasks and leave 1 task in running
    +    for (id <- Set(0, 1, 2)) {
    +      manager.handleSuccessfulTask(id, createTaskResult(id, accumUpdatesByTask(id)))
    +      assert(sched.endedTasks(id) === Success)
    +    }
    +
    +    // checkSpeculatableTasks checks that the task runtime is greater than the threshold for
    +    // speculating. Since we use a threshold of 0 for speculation, tasks need to be running for
    +    // > 0ms, so advance the clock by 1ms here.
    +    clock.advance(1)
    +    assert(manager.checkSpeculatableTasks(0))
    +    assert(sched.speculativeTasks.toSet === Set(3))
    +
    +    // Offer resource to start the speculative attempt for the running task
    +    val taskOption5 = manager.resourceOffer("exec1", "host1", NO_PREF)
    +    assert(taskOption5.isDefined)
    +    val task5 = taskOption5.get
    +    assert(task5.index === 3)
    +    assert(task5.taskId === 4)
    +    assert(task5.executorId === "exec1")
    +    assert(task5.attemptNumber === 1)
    +    sched.backend = mock(classOf[SchedulerBackend])
    +
    +    // Complete one attempt for the running task
    +    manager.handleSuccessfulTask(3, createTaskResult(3, accumUpdatesByTask(3)))
    +    // Verify that it kills other running attempt
    +    verify(sched.backend).killTask(4, "exec1", true, "another attempt succeeded")
    +    // Complete another attempt for the running task
    +    manager.handleSuccessfulTask(4, createTaskResult(3, accumUpdatesByTask(3)))
    +
    +    assert(manager.taskInfos(3).successful == true)
    +    assert(manager.taskInfos(4).killed == true)
    --- End diff --
    
    it seems the main thing you're trying to change here is what gets passed to `DAGScheduler.taskEnded`, so shouldn't you be verifying that here?


---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    **[Test build #92425 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92425/testReport)** for PR 21653 at commit [`8f7d981`](https://github.com/apache/spark/commit/8f7d98177816e11659cf79a2b28f96bd4b7173d5).


---

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


[GitHub] spark pull request #21653: [SPARK-13343] speculative tasks that didn't commi...

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

    https://github.com/apache/spark/pull/21653#discussion_r199549772
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala ---
    @@ -1371,4 +1371,64 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
         val valueSer = SparkEnv.get.serializer.newInstance()
         new DirectTaskResult[Int](valueSer.serialize(id), accumUpdates)
       }
    +
    +  test("SPARK-13343 speculative tasks that didn't commit shouldn't be marked as success") {
    +    sc = new SparkContext("local", "test")
    +    sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2"))
    +    val taskSet = FakeTask.createTaskSet(4)
    +    // Set the speculation multiplier to be 0 so speculative tasks are launched immediately
    +    sc.conf.set("spark.speculation.multiplier", "0.0")
    +    sc.conf.set("spark.speculation", "true")
    +    val clock = new ManualClock()
    +    val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock)
    +    val accumUpdatesByTask: Array[Seq[AccumulatorV2[_, _]]] = taskSet.tasks.map { task =>
    +      task.metrics.internalAccums
    +    }
    +
    +    // Offer resources for 4 tasks to start
    +    for ((k, v) <- List(
    +      "exec1" -> "host1",
    +      "exec1" -> "host1",
    +      "exec2" -> "host2",
    +      "exec2" -> "host2")) {
    --- End diff --
    
    nit: double indent the contents of the `List`


---

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


[GitHub] spark pull request #21653: [SPARK-13343] speculative tasks that didn't commi...

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

    https://github.com/apache/spark/pull/21653#discussion_r204199580
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala ---
    @@ -723,6 +723,21 @@ private[spark] class TaskSetManager(
       def handleSuccessfulTask(tid: Long, result: DirectTaskResult[_]): Unit = {
         val info = taskInfos(tid)
         val index = info.index
    +    // Check if any other attempt succeeded before this and this attempt has not been handled
    +    if (successful(index) && killedByOtherAttempt.contains(tid)) {
    +      calculatedTasks -= 1
    +
    +      val resultSizeAcc = result.accumUpdates.find(a =>
    +        a.name == Some(InternalAccumulator.RESULT_SIZE))
    +      if (resultSizeAcc.isDefined) {
    +        totalResultSize -= resultSizeAcc.get.asInstanceOf[LongAccumulator].value
    --- End diff --
    
    I agree, I dont see a better option.



---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    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 #21653: [SPARK-13343] speculative tasks that didn't commi...

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

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


---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    +1, changes look good to me. 
    
    @squito   see any problems with this approach?


---

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


[GitHub] spark pull request #21653: [SPARK-13343] speculative tasks that didn't commi...

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

    https://github.com/apache/spark/pull/21653#discussion_r204090000
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala ---
    @@ -723,6 +723,21 @@ private[spark] class TaskSetManager(
       def handleSuccessfulTask(tid: Long, result: DirectTaskResult[_]): Unit = {
         val info = taskInfos(tid)
         val index = info.index
    +    // Check if any other attempt succeeded before this and this attempt has not been handled
    +    if (successful(index) && killedByOtherAttempt.contains(tid)) {
    +      calculatedTasks -= 1
    --- End diff --
    
    comment here about cleaning up things from incremented earlier while handling it as successful


---

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


[GitHub] spark issue #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

    https://github.com/apache/spark/pull/21653
  
    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 #21653: [SPARK-13343] speculative tasks that didn't commit shoul...

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

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


---

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