@@ -22,6 +22,7 @@ import scala.collection.mutable
2222import org .apache .spark ._
2323import org .apache .spark .internal .Logging
2424import org .apache .spark .rpc .{RpcCallContext , RpcEndpoint , RpcEndpointRef , RpcEnv }
25+ import org .apache .spark .util .{RpcUtils , ThreadUtils }
2526
2627private sealed trait OutputCommitCoordinationMessage extends Serializable
2728
@@ -88,7 +89,8 @@ private[spark] class OutputCommitCoordinator(conf: SparkConf, isDriver: Boolean)
8889 val msg = AskPermissionToCommitOutput (stage, partition, attemptNumber)
8990 coordinatorRef match {
9091 case Some (endpointRef) =>
91- endpointRef.askWithRetry[Boolean ](msg)
92+ ThreadUtils .awaitResult(endpointRef.ask[Boolean ](msg),
93+ RpcUtils .askRpcTimeout(conf).duration)
9294 case None =>
9395 logError(
9496 " canCommit called after coordinator was stopped (is SparkEnv shutdown in progress)?" )
@@ -165,9 +167,18 @@ private[spark] class OutputCommitCoordinator(conf: SparkConf, isDriver: Boolean)
165167 authorizedCommitters(partition) = attemptNumber
166168 true
167169 case existingCommitter =>
168- logDebug(s " Denying attemptNumber= $attemptNumber to commit for stage= $stage, " +
169- s " partition= $partition; existingCommitter = $existingCommitter" )
170- false
170+ // Coordinator should be idempotent when receiving AskPermissionToCommit.
171+ if (existingCommitter == attemptNumber) {
172+ logWarning(s " Authorizing duplicate request to commit for " +
173+ s " attemptNumber= $attemptNumber to commit for stage= $stage, " +
174+ s " partition= $partition; existingCommitter = $existingCommitter. " +
175+ s " This can indicate dropped network traffic. " )
176+ true
177+ } else {
178+ logDebug(s " Denying attemptNumber= $attemptNumber to commit for stage= $stage, " +
179+ s " partition= $partition; existingCommitter = $existingCommitter" )
180+ false
181+ }
171182 }
172183 case None =>
173184 logDebug(s " Stage $stage has completed, so not allowing attempt number $attemptNumber of " +
0 commit comments