Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-18113] Use ask to replace askWithRetry in canCommit and make receiver idempotent. #16503

Closed
wants to merge 1 commit into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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