Skip to content

Commit

Permalink
Merge pull request apache#139 from abellina/yspark_2_1_1_YSPARK-654_p…
Browse files Browse the repository at this point in the history
…ull_SPARK-18113_can_commit_retries

[SPARK-18113] Use ask to replace askWithRetry in canCommit and make r…
  • Loading branch information
Tom Graves authored and GitHub Enterprise committed May 5, 2017
2 parents db0cc99 + 74e2ca7 commit 8654e55
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 @@ -190,6 +190,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 @@ -222,6 +228,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 8654e55

Please sign in to comment.