Github user vanzin commented on a diff in the pull request: https://github.com/apache/spark/pull/16959#discussion_r103550855 --- Diff: core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala --- @@ -195,6 +195,17 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter { sc.runJob(rdd, OutputCommitFunctions(tempDir.getAbsolutePath).callCanCommitMultipleTimes _, 0 until rdd.partitions.size) } + + test("SPARK-19631: Do not allow failed attempts to be authorized for committing") { + val stage: Int = 1 + val partition: Int = 1 + val failedAttempt: Int = 0 + outputCommitCoordinator.stageStart(stage, maxPartitionId = 1) + outputCommitCoordinator.taskCompleted(stage, partition, attemptNumber = failedAttempt, + reason = ExecutorLostFailure("0", exitCausedByApp = true, None)) + assert(!outputCommitCoordinator.canCommit(stage, partition, failedAttempt)) --- End diff -- > But what if the earlier attempt had committed? I think there's a really tricky race here that I don't know if Spark is even able to fix. Basically: - E1 starts to commit - E1 loses connectivity with the driver, still committing - E2 gets permission to commit from driver - both E1 and E2 are committing their output files Depending on how those output files are generated, you may either end up with corrupt output (i.e. output files from both executors) or task failures with partial output (E2 would fail to commit, you'd have some output from E1 in the final directory, task would be retried). I believe that both tasks in this case would have the same set of output files, so one would overwrite the other; so the real problem is if E1 "wins" the race but leaves incomplete output around.
--- 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