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/03/15 15:58:49 UTC

[GitHub] [spark] xuanyuanking opened a new pull request #24110: [SPARK-25341][Core] Support rolling back a shuffle map stage and re-generate the shuffle files

xuanyuanking opened a new pull request #24110: [SPARK-25341][Core] Support rolling back a shuffle map stage and re-generate the shuffle files
URL: https://github.com/apache/spark/pull/24110
 
 
   ## What changes were proposed in this pull request?
   
   This is a follow-up work for #22112's future improvment[1]: `Currently we can't rollback and rerun a shuffle map stage, and just fail.`All changes are summarized as follows:
   - Extend ShuffleBlockId with indeterminateAttemptId.
   - Add corresponding support for ShuffleBlockResolver, if the shuffle file generated from the indeterminate stage, its name will contain the indeterminateAttemptId, otherwise the file name just as before.
   - Add the determinate flag in TaskContext and use it in Shuffle Reader and Writer.
   - Track the indeterminate attempt id in ShuffleStatus and add register and unregister support in MapOutputTracker
   - Add the determinate flag in Stage and use it in DAGScheduler. Also consider about the cleaning work for the intermediate state for the indeterminate stage.
   
   ## How was this patch tested?
   
   - UT: Add UT for all changing code and newly added function.
   - Manual Test:
   Also providing a manual test to verify the effect:
   ```
   import scala.sys.process._
   import org.apache.spark.TaskContext
   
   val determinateStage0 = sc.parallelize(0 until 1000 * 1000 * 100, 10)
   val indeterminateStage1 = determinateStage0.repartition(200)
   val indeterminateStage2 = indeterminateStage1.repartition(200)
   val indeterminateStage3 = indeterminateStage2.repartition(100)
   val indeterminateStage4 = indeterminateStage3.repartition(300)
   val fetchFailIndeterminateStage4 = indeterminateStage4.map { x =>
   if (TaskContext.get.attemptNumber == 0 && TaskContext.get.partitionId == 190 && 
     TaskContext.get.stageAttemptNumber == 0) {
     throw new Exception("pkill -f -n java".!!)
     }
     x
   }
   val indeterminateStage5 = fetchFailIndeterminateStage4.repartition(200)
   val finalStage6 = indeterminateStage5.repartition(100).collect().distinct.length
   ``` 
   It's a simple job with multi indeterminate stage, it will get a wrong anwser while useing old Spark version like 2.2/2.3, and will be killed after #22112. With this fix, the job can retry all indeterminate stage as below screenshot and get the right result.

----------------------------------------------------------------
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