Skip to content

Commit 1d30884

Browse files
viiryaHyukjinKwon
authored andcommitted
[SPARK-31484][CORE][FLOLLOWUP] Use taskAttemptId in checkpoint filename
### What changes were proposed in this pull request? As suggested by #28255 (comment), this patch proposes to use taskAttemptId in checkpoint filename, instead of stageAttemptNumber + attemptNumber. ### Why are the changes needed? To simplify checkpoint simplified and unique. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Existing tests. Closes #28289 from viirya/SPARK-31484-followup. Authored-by: Liang-Chi Hsieh <viirya@gmail.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
1 parent 37d2e03 commit 1d30884

File tree

1 file changed

+1
-2
lines changed

1 file changed

+1
-2
lines changed

core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -199,8 +199,7 @@ private[spark] object ReliableCheckpointRDD extends Logging {
199199

200200
val finalOutputName = ReliableCheckpointRDD.checkpointFileName(ctx.partitionId())
201201
val finalOutputPath = new Path(outputDir, finalOutputName)
202-
val tempOutputPath = new Path(outputDir,
203-
s".$finalOutputName-attempt-${ctx.stageAttemptNumber()}-${ctx.attemptNumber()}")
202+
val tempOutputPath = new Path(outputDir, s".$finalOutputName-attempt-${ctx.taskAttemptId()}")
204203

205204
val bufferSize = env.conf.get(BUFFER_SIZE)
206205

0 commit comments

Comments
 (0)