Skip to content
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-48260][SQL] Disable output committer coordination in one test of ParquetIOSuite #46560

Closed
wants to merge 2 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -276,8 +276,14 @@ class HadoopMapReduceCommitProtocol(
override def commitTask(taskContext: TaskAttemptContext): TaskCommitMessage = {
val attemptId = taskContext.getTaskAttemptID
logTrace(s"Commit task ${attemptId}")
val disableCommitCoordination =
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
val disableCommitCoordination =
val disableCommitCoordinationInTest =

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Also let's add a comment about why we need to disable it in test

taskContext.getConfiguration.get("spark.test.disableCommitCoordination") == "true"
SparkHadoopMapRedUtil.commitTask(
committer, taskContext, attemptId.getJobID.getId, attemptId.getTaskID.getId)
committer,
taskContext,
attemptId.getJobID.getId,
attemptId.getTaskID.getId,
disableCommitCoordination)
new TaskCommitMessage(addedAbsPathFiles.toMap -> partitionPaths.toSet)
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,8 @@ object SparkHadoopMapRedUtil extends Logging {
committer: MapReduceOutputCommitter,
mrTaskContext: MapReduceTaskAttemptContext,
jobId: Int,
splitId: Int): Unit = {
splitId: Int,
disableCommitCoordination: Boolean): Unit = {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: shall we rename as disableCommitCoordinationInTest as well?


val mrTaskAttemptID = mrTaskContext.getTaskAttemptID

Expand Down Expand Up @@ -71,7 +72,7 @@ object SparkHadoopMapRedUtil extends Logging {
sparkConf.getBoolean("spark.hadoop.outputCommitCoordination.enabled", defaultValue = true)
}

if (shouldCoordinateWithDriver) {
if (shouldCoordinateWithDriver && !disableCommitCoordination) {
val outputCommitCoordinator = SparkEnv.get.outputCommitCoordinator
val ctx = TaskContext.get()
val canCommit = outputCommitCoordinator.canCommit(ctx.stageId(), ctx.stageAttemptNumber(),
Expand All @@ -96,4 +97,12 @@ object SparkHadoopMapRedUtil extends Logging {
logInfo(s"No need to commit output of task because needsTaskCommit=false: $mrTaskAttemptID")
}
}

def commitTask(
committer: MapReduceOutputCommitter,
mrTaskContext: MapReduceTaskAttemptContext,
jobId: Int,
splitId: Int): Unit = {
commitTask(committer, mrTaskContext, jobId, splitId, disableCommitCoordination = false)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -1207,8 +1207,10 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession
}

test("SPARK-7837 Do not close output writer twice when commitTask() fails") {
withSQLConf(SQLConf.FILE_COMMIT_PROTOCOL_CLASS.key ->
classOf[SQLHadoopMapReduceCommitProtocol].getCanonicalName) {
withSQLConf(
SQLConf.FILE_COMMIT_PROTOCOL_CLASS.key ->
classOf[SQLHadoopMapReduceCommitProtocol].getCanonicalName,
"spark.test.disableCommitCoordination" -> "true") {
// Using a output committer that always fail when committing a task, so that both
// `commitTask()` and `abortTask()` are invoked.
val extraOptions = Map[String, String](
Expand Down