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

[GitHub] [spark] cloud-fan commented on a change in pull request #24375: [SPARK-27474][CORE] avoid retrying a task failed with CommitDeniedException many times

cloud-fan commented on a change in pull request #24375: [SPARK-27474][CORE] avoid retrying a task failed with CommitDeniedException many times
URL: https://github.com/apache/spark/pull/24375#discussion_r275844168
 
 

 ##########
 File path: core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
 ##########
 @@ -870,22 +874,21 @@ private[spark] class TaskSchedulerImpl(
   }
 
   /**
-   * Marks the task has completed in all TaskSetManagers for the given stage.
+   * Marks the task has completed in the active TaskSetManager for the given stage.
    *
    * After stage failure and retry, there may be multiple TaskSetManagers for the stage.
-   * If an earlier attempt of a stage completes a task, we should ensure that the later attempts
-   * do not also submit those same tasks.  That also means that a task completion from an earlier
-   * attempt can lead to the entire stage getting marked as successful.
+   * If an earlier zombie attempt of a stage completes a task, we can ask the later active attempt
+   * to skip submitting and running the task for the same partition, to save resource. That also
+   * means that a task completion from an earlier zombie attempt can lead to the entire stage
+   * getting marked as successful.
    */
-  private[scheduler] def markPartitionCompletedInAllTaskSets(
+  private[scheduler] def markPartitionCompleted(
 
 Review comment:
   Ah thanks for pointing it out! Yes it's very similar, except that this PR sends message to the task result getter thread pool first, to avoid locking in the DAGScheduler event loop thread.
   
   I don't think we will hit a deadlock. The task result getter thread pool calls `TaskSchedulerImpl.handleSuccessfulTask`, which is synchronized too. And `handleSuccessfulTask` calls method of `TaskSetManager`, which is the same as the newly added `handlePartitionCompleted`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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