-
Notifications
You must be signed in to change notification settings - Fork 28k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-31484][Core] Add stage attempt number to temp checkpoint filename to avoid file already existing exception #28255
Conversation
@@ -642,4 +643,29 @@ class CheckpointStorageSuite extends SparkFunSuite with LocalSparkContext { | |||
assert(preferredLoc == checkpointedRDD.cachedPreferredLocations.get(partiton)) | |||
} | |||
} | |||
|
|||
test("checkpoint should not fail in retry") { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Shall we add SPARK-31484:
prefix?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ok.
Hi, @viirya . This is registered as an |
@dongjoon-hyun Yes, I think so. |
Test build #121461 has finished for PR 28255 at commit
|
retest this please |
Test build #121465 has finished for PR 28255 at commit
|
also cc @cloud-fan @HyukjinKwon |
Test build #121471 has finished for PR 28255 at commit
|
retest this please |
Test build #121474 has finished for PR 28255 at commit
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
+1, LGTM. Thank you, @viirya and @HyukjinKwon .
Merged to master for Apache Spark 3.1.0.
val tempOutputPath = | ||
new Path(outputDir, s".$finalOutputName-attempt-${ctx.attemptNumber()}") | ||
val tempOutputPath = new Path(outputDir, | ||
s".$finalOutputName-attempt-${ctx.stageAttemptNumber()}-${ctx.attemptNumber()}") |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
stageAttemptNumber
isn't unique among stages and attemptNumber
isn't unique among tasks within the same stage. So it seems that this could still lead to the file name conflict. e.g. task 0.0 from stage 0.0 could conflict with task 1.0 from stage 1.0 (different stage) and task 1.0 from stage 1.0 could conflict with task 2.0 from stage 1.0 (same stage).
I think the unique file format should be ...-stageId-stageAttemptId-taskId-taskAttemptId-...
.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
A RDD can across stages?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
No, but all attempts Id starts from 0.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Oh, I see we suffix the checkpoint path with rdd id...
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If we just want a unique file name, can we use the task id? It's unique within the Spark application.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Do you mean taskAttemptId
?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, taskAttemptId
, we also use it in the shuffle map file for making the file name unique. #24892 (comment)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
OK. Let me create a follow-up for it. Thanks.
### 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>
What changes were proposed in this pull request?
To avoid file already existing exception when creating checkpoint file, this PR proposes to add stage attempt number to temporary checkpoint file.
Why are the changes needed?
On our production clusters, we have seen checkpointing failure. The failed stage can possibly leave partial written checkpoint file, the task of retried stage to write checkpoint file could fail due to
FileAlreadyExistsException
when creating the same file, likeDoes this PR introduce any user-facing change?
Yes. Users won't see checkpoint file already existing exception after this PR.
How was this patch tested?
Add unit test.