You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by vanzin <gi...@git.apache.org> on 2017/07/11 00:16:04 UTC

[GitHub] spark pull request #18594: [SPARK-20904][core] Don't report task failures to...

GitHub user vanzin opened a pull request:

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

    [SPARK-20904][core] Don't report task failures to driver during shutdown.

    Executors run a thread pool with daemon threads to run tasks. This means
    that those threads remain active when the JVM is shutting down, meaning
    those tasks are affected by code that runs in shutdown hooks.
    
    So if a shutdown hook messes with something that the task is using (e.g.
    an HDFS connection), the task will fail and will report that failure to
    the driver. That will make the driver mark the task as failed regardless
    of what caused the executor to shut down. So, for example, if YARN pre-empted
    that executor, the driver would consider that task failed when it should
    instead ignore the failure.
    
    This change avoids reporting failures to the driver when shutdown hooks
    are executing; this fixes the YARN preemption accounting, and doesn't really
    change things much for other scenarios, other than reporting a more generic
    error ("Executor lost") when the executor shuts down unexpectedly - which
    is arguably more correct.
    
    Tested with a hacky app running on spark-shell that tried to cause failures
    only when shutdown hooks were running, verified that preemption didn't cause
    the app to fail because of task failures exceeding the threshold.


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

    $ git pull https://github.com/vanzin/spark SPARK-20904

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

    https://github.com/apache/spark/pull/18594.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 #18594
    
----
commit 76de32a22cda3edab5f6e7baa12af80112715051
Author: Marcelo Vanzin <va...@cloudera.com>
Date:   2017-07-11T00:02:18Z

    [SPARK-20904][core] Don't report task failures to driver during shutdown.
    
    Executors run a thread pool with daemon threads to run tasks. This means
    that those threads remain active when the JVM is shutting down, meaning
    those tasks are affected by code that runs in shutdown hooks.
    
    So if a shutdown hook messes with something that the task is using (e.g.
    an HDFS connection), the task will fail and will report that failure to
    the driver. That will make the driver mark the task as failed regardless
    of what caused the executor to shut down. So, for example, if YARN pre-empted
    that executor, the driver would consider that task failed when it should
    instead ignore the failure.
    
    This change avoids reporting failures to the driver when shutdown hooks
    are executing; this fixes the YARN preemption accounting, and doesn't really
    change things much for other scenarios, other than reporting a more generic
    error ("Executor lost") when the executor shuts down unexpectedly - which
    is arguably more correct.
    
    Tested with a hacky app running on spark-shell that tried to cause failures
    only when shutdown hooks were running, verified that preemption didn't cause
    the app to fail because of task failures exceeding the threshold.

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18594: [SPARK-20904][core] Don't report task failures to driver...

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

    https://github.com/apache/spark/pull/18594
  
    I'm hesitant to support the change. If we don't notify the failure to driver, the status of the failed task would not be updated, thus not rescheduled, perhaps it's not the behavior we expect to see?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18594: [SPARK-20904][core] Don't report task failures to driver...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18594: [SPARK-20904][core] Don't report task failures to driver...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18594: [SPARK-20904][core] Don't report task failures to...

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

    https://github.com/apache/spark/pull/18594#discussion_r128871196
  
    --- Diff: core/src/main/scala/org/apache/spark/executor/Executor.scala ---
    @@ -473,29 +473,36 @@ private[spark] class Executor(
               // the default uncaught exception handler, which will terminate the Executor.
               logError(s"Exception in $taskName (TID $taskId)", t)
     
    -          // Collect latest accumulator values to report back to the driver
    -          val accums: Seq[AccumulatorV2[_, _]] =
    -            if (task != null) {
    -              task.metrics.setExecutorRunTime(System.currentTimeMillis() - taskStart)
    -              task.metrics.setJvmGCTime(computeTotalGcTime() - startGCTime)
    -              task.collectAccumulatorUpdates(taskFailed = true)
    -            } else {
    -              Seq.empty
    -            }
    +          // SPARK-20904: Do not report failure to driver if if happened during shut down. Because
    +          // libraries may set up shutdown hooks that race with running tasks during shutdown,
    +          // spurious failures may occur and can result in improper accounting in the driver (e.g.
    +          // the task failure would not be ignored if the shutdown happened because of premption,
    +          // instead of an app issue).
    +          if (!ShutdownHookManager.inShutdown()) {
    --- End diff --
    
    At this point I don't think we have any information on why we're in shutdown, whether it is an app issue, the Spark executor process being killed from the command line, etc.
    
    Yes, a nice log message would be nice. Maybe, in the else clause to this if, something like logInfo(s"Not reporting failure as we are in the middle of a shutdown").


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18594: [SPARK-20904][core] Don't report task failures to...

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

    https://github.com/apache/spark/pull/18594#discussion_r128872073
  
    --- Diff: core/src/main/scala/org/apache/spark/executor/Executor.scala ---
    @@ -473,29 +473,36 @@ private[spark] class Executor(
               // the default uncaught exception handler, which will terminate the Executor.
               logError(s"Exception in $taskName (TID $taskId)", t)
     
    -          // Collect latest accumulator values to report back to the driver
    -          val accums: Seq[AccumulatorV2[_, _]] =
    -            if (task != null) {
    -              task.metrics.setExecutorRunTime(System.currentTimeMillis() - taskStart)
    -              task.metrics.setJvmGCTime(computeTotalGcTime() - startGCTime)
    -              task.collectAccumulatorUpdates(taskFailed = true)
    -            } else {
    -              Seq.empty
    -            }
    +          // SPARK-20904: Do not report failure to driver if if happened during shut down. Because
    +          // libraries may set up shutdown hooks that race with running tasks during shutdown,
    +          // spurious failures may occur and can result in improper accounting in the driver (e.g.
    +          // the task failure would not be ignored if the shutdown happened because of premption,
    +          // instead of an app issue).
    +          if (!ShutdownHookManager.inShutdown()) {
    --- End diff --
    
    Sure, I can add a log, but it's not guaranteed to be printed. During shutdown the JVM can die at any moment (only shutdown hooks run to completion, and this is not one of them)...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18594: [SPARK-20904][core] Don't report task failures to driver...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18594: [SPARK-20904][core] Don't report task failures to driver...

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

    https://github.com/apache/spark/pull/18594
  
    @squito 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18594: [SPARK-20904][core] Don't report task failures to...

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

    https://github.com/apache/spark/pull/18594#discussion_r126630789
  
    --- Diff: core/src/main/scala/org/apache/spark/executor/Executor.scala ---
    @@ -473,29 +473,36 @@ private[spark] class Executor(
               // the default uncaught exception handler, which will terminate the Executor.
               logError(s"Exception in $taskName (TID $taskId)", t)
     
    -          // Collect latest accumulator values to report back to the driver
    -          val accums: Seq[AccumulatorV2[_, _]] =
    -            if (task != null) {
    -              task.metrics.setExecutorRunTime(System.currentTimeMillis() - taskStart)
    -              task.metrics.setJvmGCTime(computeTotalGcTime() - startGCTime)
    -              task.collectAccumulatorUpdates(taskFailed = true)
    -            } else {
    -              Seq.empty
    -            }
    +          // SPARK-20904: Do not report failure to driver if if happened during shut down. Because
    +          // libraries may set up shutdown hooks that race with running tasks during shutdown,
    +          // spurious failures may occur and can result in improper accounting in the driver (e.g.
    +          // the task failure would not be ignored if the shutdown happened because of premption,
    +          // instead of an app issue).
    +          if (!ShutdownHookManager.inShutdown()) {
    --- End diff --
    
    If the shut down is caused by an app issue, do we want to report the task failure to the driver?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18594: [SPARK-20904][core] Don't report task failures to driver...

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

    https://github.com/apache/spark/pull/18594
  
    LGTM, merging to master/2.2!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18594: [SPARK-20904][core] Don't report task failures to driver...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18594: [SPARK-20904][core] Don't report task failures to driver...

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

    https://github.com/apache/spark/pull/18594
  
    **[Test build #79485 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/79485/testReport)** for PR 18594 at commit [`76de32a`](https://github.com/apache/spark/commit/76de32a22cda3edab5f6e7baa12af80112715051).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18594: [SPARK-20904][core] Don't report task failures to...

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

    https://github.com/apache/spark/pull/18594#discussion_r126753377
  
    --- Diff: core/src/main/scala/org/apache/spark/executor/Executor.scala ---
    @@ -473,29 +473,36 @@ private[spark] class Executor(
               // the default uncaught exception handler, which will terminate the Executor.
               logError(s"Exception in $taskName (TID $taskId)", t)
     
    -          // Collect latest accumulator values to report back to the driver
    -          val accums: Seq[AccumulatorV2[_, _]] =
    -            if (task != null) {
    -              task.metrics.setExecutorRunTime(System.currentTimeMillis() - taskStart)
    -              task.metrics.setJvmGCTime(computeTotalGcTime() - startGCTime)
    -              task.collectAccumulatorUpdates(taskFailed = true)
    -            } else {
    -              Seq.empty
    -            }
    +          // SPARK-20904: Do not report failure to driver if if happened during shut down. Because
    +          // libraries may set up shutdown hooks that race with running tasks during shutdown,
    +          // spurious failures may occur and can result in improper accounting in the driver (e.g.
    +          // the task failure would not be ignored if the shutdown happened because of premption,
    +          // instead of an app issue).
    +          if (!ShutdownHookManager.inShutdown()) {
    --- End diff --
    
    The task will still fail in that case, just with a different error ("Executor lost").
    
    Because the executor shutdown in that case won't be caused by the cluster manager (e.g. preemption), the task failure will still count. So aside from a different error message, everything else behaves the same in that case.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18594: [SPARK-20904][core] Don't report task failures to driver...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18594: [SPARK-20904][core] Don't report task failures to driver...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18594: [SPARK-20904][core] Don't report task failures to...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request #18594: [SPARK-20904][core] Don't report task failures to...

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

    https://github.com/apache/spark/pull/18594#discussion_r128873660
  
    --- Diff: core/src/main/scala/org/apache/spark/executor/Executor.scala ---
    @@ -473,29 +473,36 @@ private[spark] class Executor(
               // the default uncaught exception handler, which will terminate the Executor.
               logError(s"Exception in $taskName (TID $taskId)", t)
     
    -          // Collect latest accumulator values to report back to the driver
    -          val accums: Seq[AccumulatorV2[_, _]] =
    -            if (task != null) {
    -              task.metrics.setExecutorRunTime(System.currentTimeMillis() - taskStart)
    -              task.metrics.setJvmGCTime(computeTotalGcTime() - startGCTime)
    -              task.collectAccumulatorUpdates(taskFailed = true)
    -            } else {
    -              Seq.empty
    -            }
    +          // SPARK-20904: Do not report failure to driver if if happened during shut down. Because
    +          // libraries may set up shutdown hooks that race with running tasks during shutdown,
    +          // spurious failures may occur and can result in improper accounting in the driver (e.g.
    +          // the task failure would not be ignored if the shutdown happened because of premption,
    +          // instead of an app issue).
    +          if (!ShutdownHookManager.inShutdown()) {
    --- End diff --
    
    Yeah, it isn't guaranteed. I'm thinking that if this happens often enough maybe one executor will print the message, giving a clue to the user. Also it's a de-facto code comment. Yes, any daemon thread will terminate at any time at shutdown - even finishing this block isn't guaranteed. Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18594: [SPARK-20904][core] Don't report task failures to driver...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18594: [SPARK-20904][core] Don't report task failures to driver...

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

    https://github.com/apache/spark/pull/18594
  
    Here's the hacky test code for the interested:
    
    ```
    sc.parallelize(1 to 100, 100).foreach { _ =>
      var shuttingDown = false
      
      while (!shuttingDown) {
        try {
          val hook = new Thread {
            override def run() {}
          }
          // scalastyle:off runtimeaddshutdownhook
          Runtime.getRuntime.addShutdownHook(hook)
          // scalastyle:on runtimeaddshutdownhook
          Runtime.getRuntime.removeShutdownHook(hook)
          
          Thread.sleep(10)
        } catch {
          case ise: IllegalStateException => shuttingDown = true
        }
      }
    
      throw new Exception("Task failure during shutdown.")
    }
    ```
    
    Ran that in two shells, one in a low priority queue and one in a high priority one, restarting the high priority one to force several rounds of executors being killed by preemption in the low priority queue.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #18594: [SPARK-20904][core] Don't report task failures to driver...

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

    https://github.com/apache/spark/pull/18594
  
    I don't think you understand what the change is doing. The task will still fail, because the executor is dying. 
    
    The only thing that changes is the failure reason, which will now be "Executor lost", which is actually more correct (any failure caused by races in shutdown are basically because the executor is dying). That allows the driver to ignore the failure in certain cases like it already does (e.g. YARN preempting executors).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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