Github user aarondav commented on a diff in the pull request: https://github.com/apache/spark/pull/4155#discussion_r23724765 --- Diff: core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala --- @@ -106,18 +107,30 @@ class SparkHadoopWriter(@transient jobConf: JobConf) val taCtxt = getTaskContext() val cmtr = getOutputCommitter() if (cmtr.needsTaskCommit(taCtxt)) { - try { - cmtr.commitTask(taCtxt) - logInfo (taID + ": Committed") - } catch { - case e: IOException => { - logError("Error committing the output of task: " + taID.value, e) - cmtr.abortTask(taCtxt) - throw e + val outputCommitCoordinator = SparkEnv.get.outputCommitCoordinator + val canCommit = outputCommitCoordinator.canCommit(jobID, + taID.value.getTaskID().getId(), splitID, attemptID) + if (canCommit) { + try { + cmtr.commitTask(taCtxt) + logInfo (s"$taID: Committed") + } catch { + case e: IOException => { + logError("Error committing the output of task: " + taID.value, e) + cmtr.abortTask(taCtxt) + throw e + } } + } else { + val msg: String = s"$taID: Not committed because the driver did not authorize commit" + logInfo(msg) + throw new CommitDeniedException(msg, jobID, splitID, attemptID) } } else { - logInfo ("No need to commit output of task: " + taID.value) + val msg: String = s"No need to commit output of task" + + " because needsTaskCommit=false: ${taID.value}" + logInfo(msg) + throw new CommitDeniedException(msg, jobID, splitID, attemptID) --- End diff -- I think if needsTaskCommit=false, we should maintain the old behavior of not throwing an exception.
--- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- --------------------------------------------------------------------- To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org