Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/7770#discussion_r36189276
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala 
---
    @@ -773,16 +773,26 @@ class DAGScheduler(
         stage.pendingTasks.clear()
     
         // First figure out the indexes of partition ids to compute.
    -    val partitionsToCompute: Seq[Int] = {
    +    val (allPartitions: Seq[Int], partitionsToCompute: Seq[Int]) = {
           stage match {
             case stage: ShuffleMapStage =>
    -          (0 until stage.numPartitions).filter(id => 
stage.outputLocs(id).isEmpty)
    +          val allPartitions = 0 until stage.numPartitions
    +          val filteredPartitions = allPartitions.filter(id => 
stage.outputLocs(id).isEmpty)
    +          (allPartitions, filteredPartitions)
             case stage: ResultStage =>
               val job = stage.resultOfJob.get
    -          (0 until job.numPartitions).filter(id => !job.finished(id))
    +          val allPartitions = 0 until job.numPartitions
    +          val filteredPartitions = allPartitions.filter(id => 
!job.finished(id))
    +          (allPartitions, filteredPartitions)
           }
         }
     
    +    // Reset internal accumulators only if this stage is not partially 
submitted
    +    // Otherwise, we may override existing accumulator values from some 
tasks
    --- End diff --
    
    I think you are also missing the main point I was trying to make (though 
not very clearly).  I can see that is the behavior we'll have -- but I don't 
understand _why_ that is desirable.  Eg., if a user has a bunch of task 
failures, and a stage failure, why would they say: "I want to make sure that we 
only count tasks once in the case of some task failure, but for stage failure 
we should count multiple times" (already strange enough, but there is more)  
"... that is, unless I lose all my map output during the stage failure, in 
which case lets count once.  But then again, if spark doesn't *realize* that I 
lost all my map output immediately, then maybe its ok if I do count multiple 
times after all."
    
    maybe the condition you want is `stage.latestInfo.attemptId == 0`?  That 
passes the test cases you have, and it at least makes the behavior on stage 
failure & retry consistent.  But your tests also pass if you just remove 
`resetInternalAccumulators` completely, and instead just change the 
initialization to `private var _internalAccumulators: Seq[Accumulator[Long]] = 
InternalAccumulator.create()`, which I think is really just as good, since you 
only ever look at the value attached to a `StageInfo`.  (The global value will 
be some mish-mash of stages in any case if you have multiple stages running 
simultaneously.)


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

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

Reply via email to