aboutsummaryrefslogtreecommitdiff
path: root/core
diff options
context:
space:
mode:
authorjinxing <jinxing@meituan.com>2017-01-18 10:47:22 -0800
committerMarcelo Vanzin <vanzin@cloudera.com>2017-01-18 10:47:22 -0800
commit33791a8ced61d1ffa09f68033d240f874fdb1593 (patch)
tree8183db0788184cdb66d62d81956f77cb62827d7e /core
parent278fa1eb305220a85c816c948932d6af8fa619aa (diff)
downloadspark-33791a8ced61d1ffa09f68033d240f874fdb1593.tar.gz
spark-33791a8ced61d1ffa09f68033d240f874fdb1593.tar.bz2
spark-33791a8ced61d1ffa09f68033d240f874fdb1593.zip
[SPARK-18113] Use ask to replace askWithRetry in canCommit and make receiver 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 <jinxing@meituan.com> Closes #16503 from jinxing64/SPARK-18113.
Diffstat (limited to 'core')
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala19
-rw-r--r--core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala16
2 files changed, 31 insertions, 4 deletions
diff --git a/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala b/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala
index 7bed6851d0..08d220b40b 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/OutputCommitCoordinator.scala
@@ -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
@@ -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)?")
@@ -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" +
diff --git a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala
index 8c4e389e86..0c362b881d 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala
@@ -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)
+ }
}
/**
@@ -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],