Skip to content

Commit

Permalink
[SPARK-18113] Use ask to replace askWithRetry in canCommit and make r…
Browse files Browse the repository at this point in the history
…eceiver idempotent.

## What changes were proposed in this pull request?

Method canCommit sends AskPermissionToCommitOutput using askWithRetry. If timeout, it will send again. Thus AskPermissionToCommitOutput can be received multi times. Method canCommit should return the same value when called by the same attempt multi times.

In implementation before this fix, method handleAskPermissionToCommit just check if there is committer already registered, which is not enough. When worker retries AskPermissionToCommitOutput it will get CommitDeniedException, then the task will fail with reason TaskCommitDenied, which is not regarded as a task failure(SPARK-11178), so TaskScheduler will schedule this task infinitely.

In this fix, use `ask` to replace `askWithRetry` in `canCommit` and make receiver idempotent.

## How was this patch tested?

Added a new unit test to OutputCommitCoordinatorSuite.

Author: jinxing <[email protected]>

Closes apache#16503 from jinxing64/SPARK-18113.
  • Loading branch information
jinxing authored and cmonkey committed Feb 15, 2017
1 parent 5277429 commit 330f726
Show file tree
Hide file tree
Showing 2 changed files with 31 additions and 4 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import scala.collection.mutable
import org.apache.spark._
import org.apache.spark.internal.Logging
import org.apache.spark.rpc.{RpcCallContext, RpcEndpoint, RpcEndpointRef, RpcEnv}
import org.apache.spark.util.{RpcUtils, ThreadUtils}

private sealed trait OutputCommitCoordinationMessage extends Serializable

Expand Down Expand Up @@ -88,7 +89,8 @@ private[spark] class OutputCommitCoordinator(conf: SparkConf, isDriver: Boolean)
val msg = AskPermissionToCommitOutput(stage, partition, attemptNumber)
coordinatorRef match {
case Some(endpointRef) =>
endpointRef.askWithRetry[Boolean](msg)
ThreadUtils.awaitResult(endpointRef.ask[Boolean](msg),
RpcUtils.askRpcTimeout(conf).duration)
case None =>
logError(
"canCommit called after coordinator was stopped (is SparkEnv shutdown in progress)?")
Expand Down Expand Up @@ -165,9 +167,18 @@ private[spark] class OutputCommitCoordinator(conf: SparkConf, isDriver: Boolean)
authorizedCommitters(partition) = attemptNumber
true
case existingCommitter =>
logDebug(s"Denying attemptNumber=$attemptNumber to commit for stage=$stage, " +
s"partition=$partition; existingCommitter = $existingCommitter")
false
// Coordinator should be idempotent when receiving AskPermissionToCommit.
if (existingCommitter == attemptNumber) {
logWarning(s"Authorizing duplicate request to commit for " +
s"attemptNumber=$attemptNumber to commit for stage=$stage," +
s" partition=$partition; existingCommitter = $existingCommitter." +
s" This can indicate dropped network traffic.")
true
} else {
logDebug(s"Denying attemptNumber=$attemptNumber to commit for stage=$stage, " +
s"partition=$partition; existingCommitter = $existingCommitter")
false
}
}
case None =>
logDebug(s"Stage $stage has completed, so not allowing attempt number $attemptNumber of" +
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -189,6 +189,12 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter {
assert(
!outputCommitCoordinator.canCommit(stage, partition, nonAuthorizedCommitter + 3))
}

test("Duplicate calls to canCommit from the authorized committer gets idempotent responses.") {
val rdd = sc.parallelize(Seq(1), 1)
sc.runJob(rdd, OutputCommitFunctions(tempDir.getAbsolutePath).callCanCommitMultipleTimes _,
0 until rdd.partitions.size)
}
}

/**
Expand Down Expand Up @@ -221,6 +227,16 @@ private case class OutputCommitFunctions(tempDirPath: String) {
if (ctx.attemptNumber == 0) failingOutputCommitter else successfulOutputCommitter)
}

// Receiver should be idempotent for AskPermissionToCommitOutput
def callCanCommitMultipleTimes(iter: Iterator[Int]): Unit = {
val ctx = TaskContext.get()
val canCommit1 = SparkEnv.get.outputCommitCoordinator
.canCommit(ctx.stageId(), ctx.partitionId(), ctx.attemptNumber())
val canCommit2 = SparkEnv.get.outputCommitCoordinator
.canCommit(ctx.stageId(), ctx.partitionId(), ctx.attemptNumber())
assert(canCommit1 && canCommit2)
}

private def runCommitWithProvidedCommitter(
ctx: TaskContext,
iter: Iterator[Int],
Expand Down

0 comments on commit 330f726

Please sign in to comment.