-
Notifications
You must be signed in to change notification settings - Fork 28.4k
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-10381] Fix mixup of taskAttemptNumber & attemptId in OutputCommitCoordinator #8544
Conversation
@@ -122,7 +121,8 @@ object SparkHadoopMapRedUtil extends Logging { | |||
|
|||
if (shouldCoordinateWithDriver) { | |||
val outputCommitCoordinator = SparkEnv.get.outputCommitCoordinator | |||
val canCommit = outputCommitCoordinator.canCommit(jobId, splitId, attemptId) | |||
val taskAttemptNumber = TaskContext.get().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.
This is the key change in this patch; these two lines are technically the minimum diff required to fix this bug. The rest of the changes are renaming / cleanup to make the units a bit clearer.
Test build #41850 has finished for PR 8544 at commit
|
I ended up having to add some MiMa excludes. These only impact internal classes but they are RPCs that are sent over the wire. Would be good to get confirmation that this is okay from a compatibility POV. |
Test build #41873 has finished for PR 8544 at commit
|
@@ -29,7 +29,7 @@ import org.apache.spark.annotation.DeveloperApi | |||
class TaskInfo( | |||
val taskId: Long, | |||
val index: Int, | |||
val attempt: Int, | |||
val attempt: Int, // this is a task attempt number, not a globally-unique task attempt 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.
should we just rename this and add a deprecated val for backward compatibility?
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.
Yeah, that looks safe to do. Will do this when I update.
+1 on the renaming. While you are touching this, should the
|
Jenkins, retest this please. |
Test build #41900 has finished for PR 8544 at commit
|
Test build #42438 has finished for PR 8544 at commit
|
Still have to address one comment here; was just letting tests run to rule out merge conflicts. |
@squito, I took a look at the OutputCommitCoordinator itself and, as far as I can conclude, it seems to be using stageIds properly. The one area where there might be some confusion is in TaskCommitDeinedException, but in that case I think our use of job ids in log messages might actually be accurate. In either case, I think the basics fix in this patch is correct and would like to merge this version for inclusion in 1.5.1. |
Test build #42463 has finished for PR 8544 at commit
|
…port)" This reverts commit 7af9c6b.
Test build #42464 has finished for PR 8544 at commit
|
Jenkins, retest this please. |
Test build #42470 has finished for PR 8544 at commit
|
Test build #42496 has finished for PR 8544 at commit
|
retest this please |
Test build #42505 has finished for PR 8544 at commit
|
I've fixed the MiMa merge conflicts, so as soon as this latest test run passes MiMa I'm going to merge this. |
…mitCoordinator When speculative execution is enabled, consider a scenario where the authorized committer of a particular output partition fails during the OutputCommitter.commitTask() call. In this case, the OutputCommitCoordinator is supposed to release that committer's exclusive lock on committing once that task fails. However, due to a unit mismatch (we used task attempt number in one place and task attempt id in another) the lock will not be released, causing Spark to go into an infinite retry loop. This bug was masked by the fact that the OutputCommitCoordinator does not have enough end-to-end tests (the current tests use many mocks). Other factors contributing to this bug are the fact that we have many similarly-named identifiers that have different semantics but the same data types (e.g. attemptNumber and taskAttemptId, with inconsistent variable naming which makes them difficult to distinguish). This patch adds a regression test and fixes this bug by always using task attempt numbers throughout this code. Author: Josh Rosen <[email protected]> Closes #8544 from JoshRosen/SPARK-10381. (cherry picked from commit 38700ea) Signed-off-by: Josh Rosen <[email protected]>
Test build #42515 has finished for PR 8544 at commit
|
…mitCoordinator When speculative execution is enabled, consider a scenario where the authorized committer of a particular output partition fails during the OutputCommitter.commitTask() call. In this case, the OutputCommitCoordinator is supposed to release that committer's exclusive lock on committing once that task fails. However, due to a unit mismatch (we used task attempt number in one place and task attempt id in another) the lock will not be released, causing Spark to go into an infinite retry loop. This bug was masked by the fact that the OutputCommitCoordinator does not have enough end-to-end tests (the current tests use many mocks). Other factors contributing to this bug are the fact that we have many similarly-named identifiers that have different semantics but the same data types (e.g. attemptNumber and taskAttemptId, with inconsistent variable naming which makes them difficult to distinguish). This patch adds a regression test and fixes this bug by always using task attempt numbers throughout this code. Author: Josh Rosen <[email protected]> Closes apache#8544 from JoshRosen/SPARK-10381. (cherry picked from commit 38700ea) Signed-off-by: Josh Rosen <[email protected]>
…mitCoordinator When speculative execution is enabled, consider a scenario where the authorized committer of a particular output partition fails during the OutputCommitter.commitTask() call. In this case, the OutputCommitCoordinator is supposed to release that committer's exclusive lock on committing once that task fails. However, due to a unit mismatch (we used task attempt number in one place and task attempt id in another) the lock will not be released, causing Spark to go into an infinite retry loop. This bug was masked by the fact that the OutputCommitCoordinator does not have enough end-to-end tests (the current tests use many mocks). Other factors contributing to this bug are the fact that we have many similarly-named identifiers that have different semantics but the same data types (e.g. attemptNumber and taskAttemptId, with inconsistent variable naming which makes them difficult to distinguish). This patch adds a regression test and fixes this bug by always using task attempt numbers throughout this code. Author: Josh Rosen <[email protected]> Closes apache#8544 from JoshRosen/SPARK-10381. (cherry picked from commit 38700ea) Signed-off-by: Josh Rosen <[email protected]>
…mitCoordinator (branch-1.4 backport) This is a backport of #8544 to `branch-1.4` for inclusion in 1.4.2. Author: Josh Rosen <[email protected]> Closes #8789 from JoshRosen/SPARK-10381-1.4.
…mitCoordinator (branch-1.3 backport) This is a backport of #8544 to `branch-1.3` for inclusion in 1.3.2. Author: Josh Rosen <[email protected]> Closes #8790 from JoshRosen/SPARK-10381-1.3.
…mitCoordinator When speculative execution is enabled, consider a scenario where the authorized committer of a particular output partition fails during the OutputCommitter.commitTask() call. In this case, the OutputCommitCoordinator is supposed to release that committer's exclusive lock on committing once that task fails. However, due to a unit mismatch (we used task attempt number in one place and task attempt id in another) the lock will not be released, causing Spark to go into an infinite retry loop. This bug was masked by the fact that the OutputCommitCoordinator does not have enough end-to-end tests (the current tests use many mocks). Other factors contributing to this bug are the fact that we have many similarly-named identifiers that have different semantics but the same data types (e.g. attemptNumber and taskAttemptId, with inconsistent variable naming which makes them difficult to distinguish). This patch adds a regression test and fixes this bug by always using task attempt numbers throughout this code. Author: Josh Rosen <[email protected]> Closes apache#8544 from JoshRosen/SPARK-10381. (cherry picked from commit 38700ea) Signed-off-by: Josh Rosen <[email protected]> (cherry picked from commit 2bbcbc6)
When speculative execution is enabled, consider a scenario where the authorized committer of a particular output partition fails during the OutputCommitter.commitTask() call. In this case, the OutputCommitCoordinator is supposed to release that committer's exclusive lock on committing once that task fails. However, due to a unit mismatch (we used task attempt number in one place and task attempt id in another) the lock will not be released, causing Spark to go into an infinite retry loop.
This bug was masked by the fact that the OutputCommitCoordinator does not have enough end-to-end tests (the current tests use many mocks). Other factors contributing to this bug are the fact that we have many similarly-named identifiers that have different semantics but the same data types (e.g. attemptNumber and taskAttemptId, with inconsistent variable naming which makes them difficult to distinguish).
This patch adds a regression test and fixes this bug by always using task attempt numbers throughout this code.