-
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
Changes from all commits
c6220fd
a09814b
0059c95
81b86a1
7af9c6b
035d660
0d40f83
4dcb78a
edbbf6f
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -91,8 +91,7 @@ object SparkHadoopMapRedUtil extends Logging { | |
committer: MapReduceOutputCommitter, | ||
mrTaskContext: MapReduceTaskAttemptContext, | ||
jobId: Int, | ||
splitId: Int, | ||
attemptId: Int): Unit = { | ||
splitId: Int): Unit = { | ||
|
||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Here, I removed this parameter because we should always get the attempt number from the TaskContext in order to ensure that it's correct. This had a ripple-effect on a few other callsites, which I've updated. |
||
val mrTaskAttemptID = SparkHadoopUtil.get.getTaskAttemptIDFromTaskAttemptContext(mrTaskContext) | ||
|
||
|
@@ -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 commentThe 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. |
||
val canCommit = outputCommitCoordinator.canCommit(jobId, splitId, taskAttemptNumber) | ||
|
||
if (canCommit) { | ||
performCommit() | ||
|
@@ -132,7 +132,7 @@ object SparkHadoopMapRedUtil extends Logging { | |
logInfo(message) | ||
// We need to abort the task so that the driver can reschedule new attempts, if necessary | ||
committer.abortTask(mrTaskContext) | ||
throw new CommitDeniedException(message, jobId, splitId, attemptId) | ||
throw new CommitDeniedException(message, jobId, splitId, taskAttemptNumber) | ||
} | ||
} else { | ||
// Speculation is disabled or a user has chosen to manually bypass the commit coordination | ||
|
@@ -143,16 +143,4 @@ object SparkHadoopMapRedUtil extends Logging { | |
logInfo(s"No need to commit output of task because needsTaskCommit=false: $mrTaskAttemptID") | ||
} | ||
} | ||
|
||
def commitTask( | ||
committer: MapReduceOutputCommitter, | ||
mrTaskContext: MapReduceTaskAttemptContext, | ||
sparkTaskContext: TaskContext): Unit = { | ||
commitTask( | ||
committer, | ||
mrTaskContext, | ||
sparkTaskContext.stageId(), | ||
sparkTaskContext.partitionId(), | ||
sparkTaskContext.attemptNumber()) | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,68 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.spark.scheduler | ||
|
||
import org.apache.hadoop.mapred.{FileOutputCommitter, TaskAttemptContext} | ||
import org.scalatest.concurrent.Timeouts | ||
import org.scalatest.time.{Span, Seconds} | ||
|
||
import org.apache.spark.{SparkConf, SparkContext, LocalSparkContext, SparkFunSuite, TaskContext} | ||
import org.apache.spark.util.Utils | ||
|
||
/** | ||
* Integration tests for the OutputCommitCoordinator. | ||
* | ||
* See also: [[OutputCommitCoordinatorSuite]] for unit tests that use mocks. | ||
*/ | ||
class OutputCommitCoordinatorIntegrationSuite | ||
extends SparkFunSuite | ||
with LocalSparkContext | ||
with Timeouts { | ||
|
||
override def beforeAll(): Unit = { | ||
super.beforeAll() | ||
val conf = new SparkConf() | ||
.set("master", "local[2,4]") | ||
.set("spark.speculation", "true") | ||
.set("spark.hadoop.mapred.output.committer.class", | ||
classOf[ThrowExceptionOnFirstAttemptOutputCommitter].getCanonicalName) | ||
sc = new SparkContext("local[2, 4]", "test", conf) | ||
} | ||
|
||
test("exception thrown in OutputCommitter.commitTask()") { | ||
// Regression test for SPARK-10381 | ||
failAfter(Span(60, Seconds)) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The failure mode caused by this bug was an infinite loop, hence this timeout. The fact that the loop does not break after some maximum number of retries is a distinct bug which should be fixed as part of a separate patch. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Any insight on what the other bug is? Is it worth filing another minimal jira for it so we don't lose track of it? (I didn't find anything ... sorry if I just missed it.) I'm slightly concerned (perhaps just irrationally paranoid) that maybe we'll fix that bug, then regress on this bug, but this unit test won't catch it. this test doesn't have any asserts, it just checks that the job completes and doesn't throw exceptions (not that adding assertions would necessarily help us avoid that scenario either). a minimal jira would at least let us add a note to take another look at this while fixing that issue. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The other bug is due to the fact that DAGScheduler treats failures due to CommitDenied separately from other failures: they don't count towards the typical count of maximum task failures which can trigger a job failure. The correct fix is to add an upper-bound on the number of times that a commit can be denied as a last-ditch safety net to avoid infinite loop behavior. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I will follow a followup JIRA. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. makes sense, thanks Josh! There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Filed https://issues.apache.org/jira/browse/SPARK-10607 for this |
||
val tempDir = Utils.createTempDir() | ||
try { | ||
sc.parallelize(1 to 4, 2).map(_.toString).saveAsTextFile(tempDir.getAbsolutePath + "/out") | ||
} finally { | ||
Utils.deleteRecursively(tempDir) | ||
} | ||
} | ||
} | ||
} | ||
|
||
private class ThrowExceptionOnFirstAttemptOutputCommitter extends FileOutputCommitter { | ||
override def commitTask(context: TaskAttemptContext): Unit = { | ||
val ctx = TaskContext.get() | ||
if (ctx.attemptNumber < 1) { | ||
throw new java.io.FileNotFoundException("Intentional exception") | ||
} | ||
super.commitTask(context) | ||
} | ||
} |
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.
I didn't want this patch's backporting to be blocked over binary incompatibility concerns, hence my decision to use the attempt number here. The unique-within-a-SparkContext TaskAttemptIDs are Longs, which would require a change to this interface.
The parameter renaming is potentially source-incompatible. If we think that this is a concern, then I can roll back to the old incorrect name and can add a comment.