[SPARK-19450] Replace askWithRetry with askSync.

## What changes were proposed in this pull request?

`askSync` is already added in `RpcEndpointRef` (see SPARK-19347 and https://github.com/apache/spark/pull/16690#issuecomment-276850068) and `askWithRetry` is marked as deprecated.
As mentioned SPARK-18113(https://github.com/apache/spark/pull/16503#event-927953218):

>askWithRetry is basically an unneeded API, and a leftover from the akka days that doesn't make sense anymore. It's prone to cause deadlocks (exactly because it's blocking), it imposes restrictions on the caller (e.g. idempotency) and other things that people generally don't pay that much attention to when using it.

Since `askWithRetry` is just used inside spark and not in user logic. It might make sense to replace all of them with `askSync`.

## How was this patch tested?
This PR doesn't change code logic, existing unit test can cover.

Author: jinxing <jinxing@meituan.com>

Closes #16790 from jinxing64/SPARK-19450.
This commit is contained in:
jinxing 2017-02-19 04:34:07 -08:00 committed by Sean Owen
parent df3cbe3a33
commit ba8912e5f3
No known key found for this signature in database
GPG key ID: BEB3956D6717BDDC
24 changed files with 58 additions and 119 deletions

View file

@ -99,7 +99,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging
*/ */
protected def askTracker[T: ClassTag](message: Any): T = { protected def askTracker[T: ClassTag](message: Any): T = {
try { try {
trackerEndpoint.askWithRetry[T](message) trackerEndpoint.askSync[T](message)
} catch { } catch {
case e: Exception => case e: Exception =>
logError("Error communicating with MapOutputTracker", e) logError("Error communicating with MapOutputTracker", e)

View file

@ -605,7 +605,7 @@ class SparkContext(config: SparkConf) extends Logging {
Some(Utils.getThreadDump()) Some(Utils.getThreadDump())
} else { } else {
val endpointRef = env.blockManager.master.getExecutorEndpointRef(executorId).get val endpointRef = env.blockManager.master.getExecutorEndpointRef(executorId).get
Some(endpointRef.askWithRetry[Array[ThreadStackTrace]](TriggerThreadDump)) Some(endpointRef.askSync[Array[ThreadStackTrace]](TriggerThreadDump))
} }
} catch { } catch {
case e: Exception => case e: Exception =>

View file

@ -123,7 +123,7 @@ private class ClientEndpoint(
Thread.sleep(5000) Thread.sleep(5000)
logInfo("... polling master for driver state") logInfo("... polling master for driver state")
val statusResponse = val statusResponse =
activeMasterEndpoint.askWithRetry[DriverStatusResponse](RequestDriverStatus(driverId)) activeMasterEndpoint.askSync[DriverStatusResponse](RequestDriverStatus(driverId))
if (statusResponse.found) { if (statusResponse.found) {
logInfo(s"State of $driverId is ${statusResponse.state.get}") logInfo(s"State of $driverId is ${statusResponse.state.get}")
// Worker node, if present // Worker node, if present

View file

@ -1045,7 +1045,7 @@ private[deploy] object Master extends Logging {
val rpcEnv = RpcEnv.create(SYSTEM_NAME, host, port, conf, securityMgr) val rpcEnv = RpcEnv.create(SYSTEM_NAME, host, port, conf, securityMgr)
val masterEndpoint = rpcEnv.setupEndpoint(ENDPOINT_NAME, val masterEndpoint = rpcEnv.setupEndpoint(ENDPOINT_NAME,
new Master(rpcEnv, rpcEnv.address, webUiPort, securityMgr, conf)) new Master(rpcEnv, rpcEnv.address, webUiPort, securityMgr, conf))
val portsResponse = masterEndpoint.askWithRetry[BoundPortsResponse](BoundPortsRequest) val portsResponse = masterEndpoint.askSync[BoundPortsResponse](BoundPortsRequest)
(rpcEnv, portsResponse.webUIPort, portsResponse.restPort) (rpcEnv, portsResponse.webUIPort, portsResponse.restPort)
} }
} }

View file

@ -34,7 +34,7 @@ private[ui] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app")
/** Executor details for a particular application */ /** Executor details for a particular application */
def render(request: HttpServletRequest): Seq[Node] = { def render(request: HttpServletRequest): Seq[Node] = {
val appId = request.getParameter("appId") val appId = request.getParameter("appId")
val state = master.askWithRetry[MasterStateResponse](RequestMasterState) val state = master.askSync[MasterStateResponse](RequestMasterState)
val app = state.activeApps.find(_.id == appId) val app = state.activeApps.find(_.id == appId)
.getOrElse(state.completedApps.find(_.id == appId).orNull) .getOrElse(state.completedApps.find(_.id == appId).orNull)
if (app == null) { if (app == null) {

View file

@ -33,7 +33,7 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") {
private val master = parent.masterEndpointRef private val master = parent.masterEndpointRef
def getMasterState: MasterStateResponse = { def getMasterState: MasterStateResponse = {
master.askWithRetry[MasterStateResponse](RequestMasterState) master.askSync[MasterStateResponse](RequestMasterState)
} }
override def renderJson(request: HttpServletRequest): JValue = { override def renderJson(request: HttpServletRequest): JValue = {

View file

@ -71,7 +71,7 @@ private[rest] class StandaloneKillRequestServlet(masterEndpoint: RpcEndpointRef,
extends KillRequestServlet { extends KillRequestServlet {
protected def handleKill(submissionId: String): KillSubmissionResponse = { protected def handleKill(submissionId: String): KillSubmissionResponse = {
val response = masterEndpoint.askWithRetry[DeployMessages.KillDriverResponse]( val response = masterEndpoint.askSync[DeployMessages.KillDriverResponse](
DeployMessages.RequestKillDriver(submissionId)) DeployMessages.RequestKillDriver(submissionId))
val k = new KillSubmissionResponse val k = new KillSubmissionResponse
k.serverSparkVersion = sparkVersion k.serverSparkVersion = sparkVersion
@ -89,7 +89,7 @@ private[rest] class StandaloneStatusRequestServlet(masterEndpoint: RpcEndpointRe
extends StatusRequestServlet { extends StatusRequestServlet {
protected def handleStatus(submissionId: String): SubmissionStatusResponse = { protected def handleStatus(submissionId: String): SubmissionStatusResponse = {
val response = masterEndpoint.askWithRetry[DeployMessages.DriverStatusResponse]( val response = masterEndpoint.askSync[DeployMessages.DriverStatusResponse](
DeployMessages.RequestDriverStatus(submissionId)) DeployMessages.RequestDriverStatus(submissionId))
val message = response.exception.map { s"Exception from the cluster:\n" + formatException(_) } val message = response.exception.map { s"Exception from the cluster:\n" + formatException(_) }
val d = new SubmissionStatusResponse val d = new SubmissionStatusResponse
@ -174,7 +174,7 @@ private[rest] class StandaloneSubmitRequestServlet(
requestMessage match { requestMessage match {
case submitRequest: CreateSubmissionRequest => case submitRequest: CreateSubmissionRequest =>
val driverDescription = buildDriverDescription(submitRequest) val driverDescription = buildDriverDescription(submitRequest)
val response = masterEndpoint.askWithRetry[DeployMessages.SubmitDriverResponse]( val response = masterEndpoint.askSync[DeployMessages.SubmitDriverResponse](
DeployMessages.RequestSubmitDriver(driverDescription)) DeployMessages.RequestSubmitDriver(driverDescription))
val submitResponse = new CreateSubmissionResponse val submitResponse = new CreateSubmissionResponse
submitResponse.serverSparkVersion = sparkVersion submitResponse.serverSparkVersion = sparkVersion

View file

@ -34,12 +34,12 @@ private[ui] class WorkerPage(parent: WorkerWebUI) extends WebUIPage("") {
private val workerEndpoint = parent.worker.self private val workerEndpoint = parent.worker.self
override def renderJson(request: HttpServletRequest): JValue = { override def renderJson(request: HttpServletRequest): JValue = {
val workerState = workerEndpoint.askWithRetry[WorkerStateResponse](RequestWorkerState) val workerState = workerEndpoint.askSync[WorkerStateResponse](RequestWorkerState)
JsonProtocol.writeWorkerState(workerState) JsonProtocol.writeWorkerState(workerState)
} }
def render(request: HttpServletRequest): Seq[Node] = { def render(request: HttpServletRequest): Seq[Node] = {
val workerState = workerEndpoint.askWithRetry[WorkerStateResponse](RequestWorkerState) val workerState = workerEndpoint.askSync[WorkerStateResponse](RequestWorkerState)
val executorHeaders = Seq("ExecutorID", "Cores", "State", "Memory", "Job Details", "Logs") val executorHeaders = Seq("ExecutorID", "Cores", "State", "Memory", "Job Details", "Logs")
val runningExecutors = workerState.executors val runningExecutors = workerState.executors

View file

@ -199,7 +199,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging {
new SecurityManager(executorConf), new SecurityManager(executorConf),
clientMode = true) clientMode = true)
val driver = fetcher.setupEndpointRefByURI(driverUrl) val driver = fetcher.setupEndpointRefByURI(driverUrl)
val cfg = driver.askWithRetry[SparkAppConfig](RetrieveSparkAppConfig) val cfg = driver.askSync[SparkAppConfig](RetrieveSparkAppConfig)
val props = cfg.sparkProperties ++ Seq[(String, String)](("spark.app.id", appId)) val props = cfg.sparkProperties ++ Seq[(String, String)](("spark.app.id", appId))
fetcher.shutdown() fetcher.shutdown()

View file

@ -677,7 +677,7 @@ private[spark] class Executor(
val message = Heartbeat(executorId, accumUpdates.toArray, env.blockManager.blockManagerId) val message = Heartbeat(executorId, accumUpdates.toArray, env.blockManager.blockManagerId)
try { try {
val response = heartbeatReceiverRef.askWithRetry[HeartbeatResponse]( val response = heartbeatReceiverRef.askSync[HeartbeatResponse](
message, RpcTimeout(conf, "spark.executor.heartbeatInterval", "10s")) message, RpcTimeout(conf, "spark.executor.heartbeatInterval", "10s"))
if (response.reregisterBlockManager) { if (response.reregisterBlockManager) {
logInfo("Told to re-register on heartbeat") logInfo("Told to re-register on heartbeat")

View file

@ -92,64 +92,4 @@ private[spark] abstract class RpcEndpointRef(conf: SparkConf)
timeout.awaitResult(future) timeout.awaitResult(future)
} }
/**
* Send a message to the corresponding [[RpcEndpoint.receiveAndReply]] and get its result within a
* default timeout, throw a SparkException if this fails even after the default number of retries.
* The default `timeout` will be used in every trial of calling `sendWithReply`. Because this
* method retries, the message handling in the receiver side should be idempotent.
*
* Note: this is a blocking action which may cost a lot of time, so don't call it in a message
* loop of [[RpcEndpoint]].
*
* @param message the message to send
* @tparam T type of the reply message
* @return the reply message from the corresponding [[RpcEndpoint]]
*/
@deprecated("use 'askSync' instead.", "2.2.0")
def askWithRetry[T: ClassTag](message: Any): T = askWithRetry(message, defaultAskTimeout)
/**
* Send a message to the corresponding [[RpcEndpoint.receiveAndReply]] and get its result within a
* specified timeout, throw a SparkException if this fails even after the specified number of
* retries. `timeout` will be used in every trial of calling `sendWithReply`. Because this method
* retries, the message handling in the receiver side should be idempotent.
*
* Note: this is a blocking action which may cost a lot of time, so don't call it in a message
* loop of [[RpcEndpoint]].
*
* @param message the message to send
* @param timeout the timeout duration
* @tparam T type of the reply message
* @return the reply message from the corresponding [[RpcEndpoint]]
*/
@deprecated("use 'askSync' instead.", "2.2.0")
def askWithRetry[T: ClassTag](message: Any, timeout: RpcTimeout): T = {
// TODO: Consider removing multiple attempts
var attempts = 0
var lastException: Exception = null
while (attempts < maxRetries) {
attempts += 1
try {
val future = ask[T](message, timeout)
val result = timeout.awaitResult(future)
if (result == null) {
throw new SparkException("RpcEndpoint returned null")
}
return result
} catch {
case ie: InterruptedException => throw ie
case e: Exception =>
lastException = e
logWarning(s"Error sending message [message = $message] in $attempts attempts", e)
}
if (attempts < maxRetries) {
Thread.sleep(retryWaitMs)
}
}
throw new SparkException(
s"Error sending message [message = $message]", lastException)
}
} }

View file

@ -232,7 +232,7 @@ class DAGScheduler(
accumUpdates: Array[(Long, Int, Int, Seq[AccumulableInfo])], accumUpdates: Array[(Long, Int, Int, Seq[AccumulableInfo])],
blockManagerId: BlockManagerId): Boolean = { blockManagerId: BlockManagerId): Boolean = {
listenerBus.post(SparkListenerExecutorMetricsUpdate(execId, accumUpdates)) listenerBus.post(SparkListenerExecutorMetricsUpdate(execId, accumUpdates))
blockManagerMaster.driverEndpoint.askWithRetry[Boolean]( blockManagerMaster.driverEndpoint.askSync[Boolean](
BlockManagerHeartbeat(blockManagerId), new RpcTimeout(600 seconds, "BlockManagerHeartbeat")) BlockManagerHeartbeat(blockManagerId), new RpcTimeout(600 seconds, "BlockManagerHeartbeat"))
} }

View file

@ -372,7 +372,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
try { try {
if (driverEndpoint != null) { if (driverEndpoint != null) {
logInfo("Shutting down all executors") logInfo("Shutting down all executors")
driverEndpoint.askWithRetry[Boolean](StopExecutors) driverEndpoint.askSync[Boolean](StopExecutors)
} }
} catch { } catch {
case e: Exception => case e: Exception =>
@ -384,7 +384,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
stopExecutors() stopExecutors()
try { try {
if (driverEndpoint != null) { if (driverEndpoint != null) {
driverEndpoint.askWithRetry[Boolean](StopDriver) driverEndpoint.askSync[Boolean](StopDriver)
} }
} catch { } catch {
case e: Exception => case e: Exception =>

View file

@ -60,7 +60,7 @@ class BlockManagerMaster(
maxMemSize: Long, maxMemSize: Long,
slaveEndpoint: RpcEndpointRef): BlockManagerId = { slaveEndpoint: RpcEndpointRef): BlockManagerId = {
logInfo(s"Registering BlockManager $blockManagerId") logInfo(s"Registering BlockManager $blockManagerId")
val updatedId = driverEndpoint.askWithRetry[BlockManagerId]( val updatedId = driverEndpoint.askSync[BlockManagerId](
RegisterBlockManager(blockManagerId, maxMemSize, slaveEndpoint)) RegisterBlockManager(blockManagerId, maxMemSize, slaveEndpoint))
logInfo(s"Registered BlockManager $updatedId") logInfo(s"Registered BlockManager $updatedId")
updatedId updatedId
@ -72,7 +72,7 @@ class BlockManagerMaster(
storageLevel: StorageLevel, storageLevel: StorageLevel,
memSize: Long, memSize: Long,
diskSize: Long): Boolean = { diskSize: Long): Boolean = {
val res = driverEndpoint.askWithRetry[Boolean]( val res = driverEndpoint.askSync[Boolean](
UpdateBlockInfo(blockManagerId, blockId, storageLevel, memSize, diskSize)) UpdateBlockInfo(blockManagerId, blockId, storageLevel, memSize, diskSize))
logDebug(s"Updated info of block $blockId") logDebug(s"Updated info of block $blockId")
res res
@ -80,12 +80,12 @@ class BlockManagerMaster(
/** Get locations of the blockId from the driver */ /** Get locations of the blockId from the driver */
def getLocations(blockId: BlockId): Seq[BlockManagerId] = { def getLocations(blockId: BlockId): Seq[BlockManagerId] = {
driverEndpoint.askWithRetry[Seq[BlockManagerId]](GetLocations(blockId)) driverEndpoint.askSync[Seq[BlockManagerId]](GetLocations(blockId))
} }
/** Get locations of multiple blockIds from the driver */ /** Get locations of multiple blockIds from the driver */
def getLocations(blockIds: Array[BlockId]): IndexedSeq[Seq[BlockManagerId]] = { def getLocations(blockIds: Array[BlockId]): IndexedSeq[Seq[BlockManagerId]] = {
driverEndpoint.askWithRetry[IndexedSeq[Seq[BlockManagerId]]]( driverEndpoint.askSync[IndexedSeq[Seq[BlockManagerId]]](
GetLocationsMultipleBlockIds(blockIds)) GetLocationsMultipleBlockIds(blockIds))
} }
@ -99,11 +99,11 @@ class BlockManagerMaster(
/** Get ids of other nodes in the cluster from the driver */ /** Get ids of other nodes in the cluster from the driver */
def getPeers(blockManagerId: BlockManagerId): Seq[BlockManagerId] = { def getPeers(blockManagerId: BlockManagerId): Seq[BlockManagerId] = {
driverEndpoint.askWithRetry[Seq[BlockManagerId]](GetPeers(blockManagerId)) driverEndpoint.askSync[Seq[BlockManagerId]](GetPeers(blockManagerId))
} }
def getExecutorEndpointRef(executorId: String): Option[RpcEndpointRef] = { def getExecutorEndpointRef(executorId: String): Option[RpcEndpointRef] = {
driverEndpoint.askWithRetry[Option[RpcEndpointRef]](GetExecutorEndpointRef(executorId)) driverEndpoint.askSync[Option[RpcEndpointRef]](GetExecutorEndpointRef(executorId))
} }
/** /**
@ -111,12 +111,12 @@ class BlockManagerMaster(
* blocks that the driver knows about. * blocks that the driver knows about.
*/ */
def removeBlock(blockId: BlockId) { def removeBlock(blockId: BlockId) {
driverEndpoint.askWithRetry[Boolean](RemoveBlock(blockId)) driverEndpoint.askSync[Boolean](RemoveBlock(blockId))
} }
/** Remove all blocks belonging to the given RDD. */ /** Remove all blocks belonging to the given RDD. */
def removeRdd(rddId: Int, blocking: Boolean) { def removeRdd(rddId: Int, blocking: Boolean) {
val future = driverEndpoint.askWithRetry[Future[Seq[Int]]](RemoveRdd(rddId)) val future = driverEndpoint.askSync[Future[Seq[Int]]](RemoveRdd(rddId))
future.onFailure { future.onFailure {
case e: Exception => case e: Exception =>
logWarning(s"Failed to remove RDD $rddId - ${e.getMessage}", e) logWarning(s"Failed to remove RDD $rddId - ${e.getMessage}", e)
@ -128,7 +128,7 @@ class BlockManagerMaster(
/** Remove all blocks belonging to the given shuffle. */ /** Remove all blocks belonging to the given shuffle. */
def removeShuffle(shuffleId: Int, blocking: Boolean) { def removeShuffle(shuffleId: Int, blocking: Boolean) {
val future = driverEndpoint.askWithRetry[Future[Seq[Boolean]]](RemoveShuffle(shuffleId)) val future = driverEndpoint.askSync[Future[Seq[Boolean]]](RemoveShuffle(shuffleId))
future.onFailure { future.onFailure {
case e: Exception => case e: Exception =>
logWarning(s"Failed to remove shuffle $shuffleId - ${e.getMessage}", e) logWarning(s"Failed to remove shuffle $shuffleId - ${e.getMessage}", e)
@ -140,7 +140,7 @@ class BlockManagerMaster(
/** Remove all blocks belonging to the given broadcast. */ /** Remove all blocks belonging to the given broadcast. */
def removeBroadcast(broadcastId: Long, removeFromMaster: Boolean, blocking: Boolean) { def removeBroadcast(broadcastId: Long, removeFromMaster: Boolean, blocking: Boolean) {
val future = driverEndpoint.askWithRetry[Future[Seq[Int]]]( val future = driverEndpoint.askSync[Future[Seq[Int]]](
RemoveBroadcast(broadcastId, removeFromMaster)) RemoveBroadcast(broadcastId, removeFromMaster))
future.onFailure { future.onFailure {
case e: Exception => case e: Exception =>
@ -159,11 +159,11 @@ class BlockManagerMaster(
* amount of remaining memory. * amount of remaining memory.
*/ */
def getMemoryStatus: Map[BlockManagerId, (Long, Long)] = { def getMemoryStatus: Map[BlockManagerId, (Long, Long)] = {
driverEndpoint.askWithRetry[Map[BlockManagerId, (Long, Long)]](GetMemoryStatus) driverEndpoint.askSync[Map[BlockManagerId, (Long, Long)]](GetMemoryStatus)
} }
def getStorageStatus: Array[StorageStatus] = { def getStorageStatus: Array[StorageStatus] = {
driverEndpoint.askWithRetry[Array[StorageStatus]](GetStorageStatus) driverEndpoint.askSync[Array[StorageStatus]](GetStorageStatus)
} }
/** /**
@ -184,7 +184,7 @@ class BlockManagerMaster(
* master endpoint for a response to a prior message. * master endpoint for a response to a prior message.
*/ */
val response = driverEndpoint. val response = driverEndpoint.
askWithRetry[Map[BlockManagerId, Future[Option[BlockStatus]]]](msg) askSync[Map[BlockManagerId, Future[Option[BlockStatus]]]](msg)
val (blockManagerIds, futures) = response.unzip val (blockManagerIds, futures) = response.unzip
implicit val sameThread = ThreadUtils.sameThread implicit val sameThread = ThreadUtils.sameThread
val cbf = val cbf =
@ -214,7 +214,7 @@ class BlockManagerMaster(
filter: BlockId => Boolean, filter: BlockId => Boolean,
askSlaves: Boolean): Seq[BlockId] = { askSlaves: Boolean): Seq[BlockId] = {
val msg = GetMatchingBlockIds(filter, askSlaves) val msg = GetMatchingBlockIds(filter, askSlaves)
val future = driverEndpoint.askWithRetry[Future[Seq[BlockId]]](msg) val future = driverEndpoint.askSync[Future[Seq[BlockId]]](msg)
timeout.awaitResult(future) timeout.awaitResult(future)
} }
@ -223,7 +223,7 @@ class BlockManagerMaster(
* since they are not reported the master. * since they are not reported the master.
*/ */
def hasCachedBlocks(executorId: String): Boolean = { def hasCachedBlocks(executorId: String): Boolean = {
driverEndpoint.askWithRetry[Boolean](HasCachedBlocks(executorId)) driverEndpoint.askSync[Boolean](HasCachedBlocks(executorId))
} }
/** Stop the driver endpoint, called only on the Spark driver node */ /** Stop the driver endpoint, called only on the Spark driver node */
@ -237,7 +237,7 @@ class BlockManagerMaster(
/** Send a one-way message to the master endpoint, to which we expect it to reply with true. */ /** Send a one-way message to the master endpoint, to which we expect it to reply with true. */
private def tell(message: Any) { private def tell(message: Any) {
if (!driverEndpoint.askWithRetry[Boolean](message)) { if (!driverEndpoint.askSync[Boolean](message)) {
throw new SparkException("BlockManagerMasterEndpoint returned false, expected true.") throw new SparkException("BlockManagerMasterEndpoint returned false, expected true.")
} }
} }

View file

@ -547,7 +547,7 @@ class StandaloneDynamicAllocationSuite
/** Get the Master state */ /** Get the Master state */
private def getMasterState: MasterStateResponse = { private def getMasterState: MasterStateResponse = {
master.self.askWithRetry[MasterStateResponse](RequestMasterState) master.self.askSync[MasterStateResponse](RequestMasterState)
} }
/** Get the applications that are active from Master */ /** Get the applications that are active from Master */
@ -620,7 +620,7 @@ class StandaloneDynamicAllocationSuite
when(endpointRef.address).thenReturn(mockAddress) when(endpointRef.address).thenReturn(mockAddress)
val message = RegisterExecutor(id, endpointRef, "localhost", 10, Map.empty) val message = RegisterExecutor(id, endpointRef, "localhost", 10, Map.empty)
val backend = sc.schedulerBackend.asInstanceOf[CoarseGrainedSchedulerBackend] val backend = sc.schedulerBackend.asInstanceOf[CoarseGrainedSchedulerBackend]
backend.driverEndpoint.askWithRetry[Boolean](message) backend.driverEndpoint.askSync[Boolean](message)
} }
} }

View file

@ -171,7 +171,7 @@ class AppClientSuite
/** Get the Master state */ /** Get the Master state */
private def getMasterState: MasterStateResponse = { private def getMasterState: MasterStateResponse = {
master.self.askWithRetry[MasterStateResponse](RequestMasterState) master.self.askSync[MasterStateResponse](RequestMasterState)
} }
/** Get the applications that are active from Master */ /** Get the applications that are active from Master */

View file

@ -432,7 +432,7 @@ class MasterSuite extends SparkFunSuite
val master = makeMaster() val master = makeMaster()
master.rpcEnv.setupEndpoint(Master.ENDPOINT_NAME, master) master.rpcEnv.setupEndpoint(Master.ENDPOINT_NAME, master)
eventually(timeout(10.seconds)) { eventually(timeout(10.seconds)) {
val masterState = master.self.askWithRetry[MasterStateResponse](RequestMasterState) val masterState = master.self.askSync[MasterStateResponse](RequestMasterState)
assert(masterState.status === RecoveryState.ALIVE, "Master is not alive") assert(masterState.status === RecoveryState.ALIVE, "Master is not alive")
} }

View file

@ -118,8 +118,8 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll {
} }
} }
val rpcEndpointRef = env.setupEndpoint("send-ref", endpoint) val rpcEndpointRef = env.setupEndpoint("send-ref", endpoint)
val newRpcEndpointRef = rpcEndpointRef.askWithRetry[RpcEndpointRef]("Hello") val newRpcEndpointRef = rpcEndpointRef.askSync[RpcEndpointRef]("Hello")
val reply = newRpcEndpointRef.askWithRetry[String]("Echo") val reply = newRpcEndpointRef.askSync[String]("Echo")
assert("Echo" === reply) assert("Echo" === reply)
} }
@ -132,7 +132,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll {
context.reply(msg) context.reply(msg)
} }
}) })
val reply = rpcEndpointRef.askWithRetry[String]("hello") val reply = rpcEndpointRef.askSync[String]("hello")
assert("hello" === reply) assert("hello" === reply)
} }
@ -150,7 +150,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll {
// Use anotherEnv to find out the RpcEndpointRef // Use anotherEnv to find out the RpcEndpointRef
val rpcEndpointRef = anotherEnv.setupEndpointRef(env.address, "ask-remotely") val rpcEndpointRef = anotherEnv.setupEndpointRef(env.address, "ask-remotely")
try { try {
val reply = rpcEndpointRef.askWithRetry[String]("hello") val reply = rpcEndpointRef.askSync[String]("hello")
assert("hello" === reply) assert("hello" === reply)
} finally { } finally {
anotherEnv.shutdown() anotherEnv.shutdown()
@ -177,14 +177,13 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll {
// Use anotherEnv to find out the RpcEndpointRef // Use anotherEnv to find out the RpcEndpointRef
val rpcEndpointRef = anotherEnv.setupEndpointRef(env.address, "ask-timeout") val rpcEndpointRef = anotherEnv.setupEndpointRef(env.address, "ask-timeout")
try { try {
// Any exception thrown in askWithRetry is wrapped with a SparkException and set as the cause val e = intercept[RpcTimeoutException] {
val e = intercept[SparkException] { rpcEndpointRef.askSync[String]("hello", new RpcTimeout(1 millis, shortProp))
rpcEndpointRef.askWithRetry[String]("hello", new RpcTimeout(1 millis, shortProp))
} }
// The SparkException cause should be a RpcTimeoutException with message indicating the // The SparkException cause should be a RpcTimeoutException with message indicating the
// controlling timeout property // controlling timeout property
assert(e.getCause.isInstanceOf[RpcTimeoutException]) assert(e.isInstanceOf[RpcTimeoutException])
assert(e.getCause.getMessage.contains(shortProp)) assert(e.getMessage.contains(shortProp))
} finally { } finally {
anotherEnv.shutdown() anotherEnv.shutdown()
anotherEnv.awaitTermination() anotherEnv.awaitTermination()
@ -677,7 +676,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll {
} }
}) })
val rpcEndpointRef = remoteEnv.setupEndpointRef(localEnv.address, "ask-authentication") val rpcEndpointRef = remoteEnv.setupEndpointRef(localEnv.address, "ask-authentication")
val reply = rpcEndpointRef.askWithRetry[String]("hello") val reply = rpcEndpointRef.askSync[String]("hello")
assert("hello" === reply) assert("hello" === reply)
} finally { } finally {
localEnv.shutdown() localEnv.shutdown()
@ -894,7 +893,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll {
val ref = anotherEnv.setupEndpointRef(env.address, "SPARK-14699") val ref = anotherEnv.setupEndpointRef(env.address, "SPARK-14699")
// Make sure the connect is set up // Make sure the connect is set up
assert(ref.askWithRetry[String]("hello") === "hello") assert(ref.askSync[String]("hello") === "hello")
anotherEnv.shutdown() anotherEnv.shutdown()
anotherEnv.awaitTermination() anotherEnv.awaitTermination()

View file

@ -394,7 +394,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
master.removeExecutor(store.blockManagerId.executorId) master.removeExecutor(store.blockManagerId.executorId)
assert(master.getLocations("a1").size == 0, "a1 was not removed from master") assert(master.getLocations("a1").size == 0, "a1 was not removed from master")
val reregister = !master.driverEndpoint.askWithRetry[Boolean]( val reregister = !master.driverEndpoint.askSync[Boolean](
BlockManagerHeartbeat(store.blockManagerId)) BlockManagerHeartbeat(store.blockManagerId))
assert(reregister == true) assert(reregister == true)
} }

View file

@ -442,7 +442,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite
backend.stop() backend.stop()
// Any method of the backend involving sending messages to the driver endpoint should not // Any method of the backend involving sending messages to the driver endpoint should not
// be called after the backend is stopped. // be called after the backend is stopped.
verify(driverEndpoint, never()).askWithRetry(isA(classOf[RemoveExecutor]))(any[ClassTag[_]]) verify(driverEndpoint, never()).askSync(isA(classOf[RemoveExecutor]))(any[ClassTag[_]])
} }
test("mesos supports spark.executor.uri") { test("mesos supports spark.executor.uri") {

View file

@ -101,7 +101,7 @@ private[yarn] class YarnAllocator(
* @see SPARK-12864 * @see SPARK-12864
*/ */
private var executorIdCounter: Int = private var executorIdCounter: Int =
driverRef.askWithRetry[Int](RetrieveLastAllocatedExecutorId) driverRef.askSync[Int](RetrieveLastAllocatedExecutorId)
// Queue to store the timestamp of failed executors // Queue to store the timestamp of failed executors
private val failedExecutorsTimeStamps = new Queue[Long]() private val failedExecutorsTimeStamps = new Queue[Long]()

View file

@ -88,21 +88,21 @@ class StateStoreCoordinatorRef private(rpcEndpointRef: RpcEndpointRef) {
/** Verify whether the given executor has the active instance of a state store */ /** Verify whether the given executor has the active instance of a state store */
private[state] def verifyIfInstanceActive(storeId: StateStoreId, executorId: String): Boolean = { private[state] def verifyIfInstanceActive(storeId: StateStoreId, executorId: String): Boolean = {
rpcEndpointRef.askWithRetry[Boolean](VerifyIfInstanceActive(storeId, executorId)) rpcEndpointRef.askSync[Boolean](VerifyIfInstanceActive(storeId, executorId))
} }
/** Get the location of the state store */ /** Get the location of the state store */
private[state] def getLocation(storeId: StateStoreId): Option[String] = { private[state] def getLocation(storeId: StateStoreId): Option[String] = {
rpcEndpointRef.askWithRetry[Option[String]](GetLocation(storeId)) rpcEndpointRef.askSync[Option[String]](GetLocation(storeId))
} }
/** Deactivate instances related to a set of operator */ /** Deactivate instances related to a set of operator */
private[state] def deactivateInstances(storeRootLocation: String): Unit = { private[state] def deactivateInstances(storeRootLocation: String): Unit = {
rpcEndpointRef.askWithRetry[Boolean](DeactivateInstances(storeRootLocation)) rpcEndpointRef.askSync[Boolean](DeactivateInstances(storeRootLocation))
} }
private[state] def stop(): Unit = { private[state] def stop(): Unit = {
rpcEndpointRef.askWithRetry[Boolean](StopCoordinator) rpcEndpointRef.askSync[Boolean](StopCoordinator)
} }
} }

View file

@ -188,13 +188,13 @@ private[streaming] class ReceiverSupervisorImpl(
override protected def onReceiverStart(): Boolean = { override protected def onReceiverStart(): Boolean = {
val msg = RegisterReceiver( val msg = RegisterReceiver(
streamId, receiver.getClass.getSimpleName, host, executorId, endpoint) streamId, receiver.getClass.getSimpleName, host, executorId, endpoint)
trackerEndpoint.askWithRetry[Boolean](msg) trackerEndpoint.askSync[Boolean](msg)
} }
override protected def onReceiverStop(message: String, error: Option[Throwable]) { override protected def onReceiverStop(message: String, error: Option[Throwable]) {
logInfo("Deregistering receiver " + streamId) logInfo("Deregistering receiver " + streamId)
val errorString = error.map(Throwables.getStackTraceAsString).getOrElse("") val errorString = error.map(Throwables.getStackTraceAsString).getOrElse("")
trackerEndpoint.askWithRetry[Boolean](DeregisterReceiver(streamId, message, errorString)) trackerEndpoint.askSync[Boolean](DeregisterReceiver(streamId, message, errorString))
logInfo("Stopped receiver " + streamId) logInfo("Stopped receiver " + streamId)
} }

View file

@ -170,7 +170,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false
trackerState = Stopping trackerState = Stopping
if (!skipReceiverLaunch) { if (!skipReceiverLaunch) {
// Send the stop signal to all the receivers // Send the stop signal to all the receivers
endpoint.askWithRetry[Boolean](StopAllReceivers) endpoint.askSync[Boolean](StopAllReceivers)
// Wait for the Spark job that runs the receivers to be over // Wait for the Spark job that runs the receivers to be over
// That is, for the receivers to quit gracefully. // That is, for the receivers to quit gracefully.
@ -183,7 +183,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false
} }
// Check if all the receivers have been deregistered or not // Check if all the receivers have been deregistered or not
val receivers = endpoint.askWithRetry[Seq[Int]](AllReceiverIds) val receivers = endpoint.askSync[Seq[Int]](AllReceiverIds)
if (receivers.nonEmpty) { if (receivers.nonEmpty) {
logWarning("Not all of the receivers have deregistered, " + receivers) logWarning("Not all of the receivers have deregistered, " + receivers)
} else { } else {
@ -249,7 +249,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false
*/ */
def allocatedExecutors(): Map[Int, Option[String]] = synchronized { def allocatedExecutors(): Map[Int, Option[String]] = synchronized {
if (isTrackerStarted) { if (isTrackerStarted) {
endpoint.askWithRetry[Map[Int, ReceiverTrackingInfo]](GetAllReceiverInfo).mapValues { endpoint.askSync[Map[Int, ReceiverTrackingInfo]](GetAllReceiverInfo).mapValues {
_.runningExecutor.map { _.runningExecutor.map {
_.executorId _.executorId
} }