Skip to content

Commit

Permalink
[SPARK-32850][CORE][K8S] Simplify the RPC message flow of decommission
Browse files Browse the repository at this point in the history
### What changes were proposed in this pull request?

This PR cleans up the RPC message flow among the multiple decommission use cases, it includes changes:

* Keep `Worker`'s decommission status be consistent between the case where decommission starts from `Worker` and the case where decommission starts from the `MasterWebUI`: sending `DecommissionWorker` from `Master` to `Worker` in the latter case.

* Change from two-way communication to one-way communication when notifying decommission between driver and executor: it's obviously unnecessary for the executor to acknowledge the decommission status to the driver since the decommission request is from the driver. And it's same in reverse.

* Only send one message instead of two(`DecommissionSelf`/`DecommissionBlockManager`) when decommission the executor: executor and `BlockManager` are in the same JVM.

* Clean up codes around here.

### Why are the changes needed?

Before:

<img width="1948" alt="WeChat56c00cc34d9785a67a544dca036d49da" src="https://user-images.githubusercontent.com/16397174/92850308-dc461c80-f41e-11ea-8ac0-287825f4e0c4.png">

After:
<img width="1968" alt="WeChat05f7afb017e3f0132394c5e54245e49e" src="https://user-images.githubusercontent.com/16397174/93189571-de88dd80-f774-11ea-9300-1943920aa27d.png">

(Note the diagrams only counts those RPC calls that needed to go through the network. Local RPC calls are not counted here.)

After this change, We reduced 6 original RPC calls and added one more RPC call for keeping the consistent decommission status for the Worker. And the RPC flow becomes more clear.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Updated existing tests.

Closes apache#29817 from Ngone51/simplify-decommission-rpc.

Authored-by: yi.wu <[email protected]>
Signed-off-by: HyukjinKwon <[email protected]>
  • Loading branch information
Ngone51 authored and HyukjinKwon committed Oct 23, 2020
1 parent 87b32f6 commit edeecad
Show file tree
Hide file tree
Showing 19 changed files with 257 additions and 166 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -91,11 +91,13 @@ private[spark] trait ExecutorAllocationClient {
* @param executorsAndDecomInfo identifiers of executors & decom info.
* @param adjustTargetNumExecutors whether the target number of executors will be adjusted down
* after these executors have been decommissioned.
* @param triggeredByExecutor whether the decommission is triggered at executor.
* @return the ids of the executors acknowledged by the cluster manager to be removed.
*/
def decommissionExecutors(
executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
adjustTargetNumExecutors: Boolean): Seq[String] = {
executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
adjustTargetNumExecutors: Boolean,
triggeredByExecutor: Boolean): Seq[String] = {
killExecutors(executorsAndDecomInfo.map(_._1),
adjustTargetNumExecutors,
countFailures = false)
Expand All @@ -109,14 +111,21 @@ private[spark] trait ExecutorAllocationClient {
* @param executorId identifiers of executor to decommission
* @param decommissionInfo information about the decommission (reason, host loss)
* @param adjustTargetNumExecutors if we should adjust the target number of executors.
* @param triggeredByExecutor whether the decommission is triggered at executor.
* (TODO: add a new type like `ExecutorDecommissionInfo` for the
* case where executor is decommissioned at executor first, so we
* don't need this extra parameter.)
* @return whether the request is acknowledged by the cluster manager.
*/
final def decommissionExecutor(executorId: String,
final def decommissionExecutor(
executorId: String,
decommissionInfo: ExecutorDecommissionInfo,
adjustTargetNumExecutors: Boolean): Boolean = {
adjustTargetNumExecutors: Boolean,
triggeredByExecutor: Boolean = false): Boolean = {
val decommissionedExecutors = decommissionExecutors(
Array((executorId, decommissionInfo)),
adjustTargetNumExecutors = adjustTargetNumExecutors)
adjustTargetNumExecutors = adjustTargetNumExecutors,
triggeredByExecutor = triggeredByExecutor)
decommissionedExecutors.nonEmpty && decommissionedExecutors(0).equals(executorId)
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -580,7 +580,10 @@ private[spark] class ExecutorAllocationManager(
if (decommissionEnabled) {
val executorIdsWithoutHostLoss = executorIdsToBeRemoved.toSeq.map(
id => (id, ExecutorDecommissionInfo("spark scale down"))).toArray
client.decommissionExecutors(executorIdsWithoutHostLoss, adjustTargetNumExecutors = false)
client.decommissionExecutors(
executorIdsWithoutHostLoss,
adjustTargetNumExecutors = false,
triggeredByExecutor = false)
} else {
client.killExecutors(executorIdsToBeRemoved.toSeq, adjustTargetNumExecutors = false,
countFailures = false, force = false)
Expand Down
32 changes: 27 additions & 5 deletions core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
Original file line number Diff line number Diff line change
Expand Up @@ -61,13 +61,35 @@ private[deploy] object DeployMessages {
}

/**
* An internal message that used by Master itself, in order to handle the
* `DecommissionWorkersOnHosts` request from `MasterWebUI` asynchronously.
* @param ids A collection of Worker ids, which should be decommissioned.
*/
case class DecommissionWorkers(ids: Seq[String]) extends DeployMessage

/**
* A message that sent from Master to Worker to decommission the Worker.
* It's used for the case where decommission is triggered at MasterWebUI.
*
* Note that decommission a Worker will cause all the executors on that Worker
* to be decommissioned as well.
*/
object DecommissionWorker extends DeployMessage

/**
* A message that sent by the Worker to itself when it receives PWR signal,
* indicating the Worker starts to decommission.
*/
object WorkerSigPWRReceived extends DeployMessage

/**
* A message sent from Worker to Master to tell Master that the Worker has started
* decommissioning. It's used for the case where decommission is triggered at Worker.
*
* @param id the worker id
* @param worker the worker endpoint ref
* @param workerRef the worker endpoint ref
*/
case class WorkerDecommission(
id: String,
worker: RpcEndpointRef)
extends DeployMessage
case class WorkerDecommissioning(id: String, workerRef: RpcEndpointRef) extends DeployMessage

case class ExecutorStateChanged(
appId: String,
Expand Down
21 changes: 15 additions & 6 deletions core/src/main/scala/org/apache/spark/deploy/master/Master.scala
Original file line number Diff line number Diff line change
Expand Up @@ -245,15 +245,27 @@ private[deploy] class Master(
logError("Leadership has been revoked -- master shutting down.")
System.exit(0)

case WorkerDecommission(id, workerRef) =>
logInfo("Recording worker %s decommissioning".format(id))
case WorkerDecommissioning(id, workerRef) =>
if (state == RecoveryState.STANDBY) {
workerRef.send(MasterInStandby)
} else {
// We use foreach since get gives us an option and we can skip the failures.
idToWorker.get(id).foreach(decommissionWorker)
}

case DecommissionWorkers(ids) =>
// The caller has already checked the state when handling DecommissionWorkersOnHosts,
// so it should not be the STANDBY
assert(state != RecoveryState.STANDBY)
ids.foreach ( id =>
// We use foreach since get gives us an option and we can skip the failures.
idToWorker.get(id).foreach { w =>
decommissionWorker(w)
// Also send a message to the worker node to notify.
w.endpoint.send(DecommissionWorker)
}
)

case RegisterWorker(
id, workerHost, workerPort, workerRef, cores, memory, workerWebUiUrl,
masterAddress, resources) =>
Expand Down Expand Up @@ -891,10 +903,7 @@ private[deploy] class Master(
logInfo(s"Decommissioning the workers with host:ports ${workersToRemoveHostPorts}")

// The workers are removed async to avoid blocking the receive loop for the entire batch
workersToRemove.foreach(wi => {
logInfo(s"Sending the worker decommission to ${wi.id} and ${wi.endpoint}")
self.send(WorkerDecommission(wi.id, wi.endpoint))
})
self.send(DecommissionWorkers(workersToRemove.map(_.id).toSeq))

// Return the count of workers actually removed
workersToRemove.size
Expand Down
29 changes: 19 additions & 10 deletions core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,10 @@ private[deploy] class Worker(
if (conf.get(config.DECOMMISSION_ENABLED)) {
logInfo("Registering SIGPWR handler to trigger decommissioning.")
SignalUtils.register("PWR", "Failed to register SIGPWR handler - " +
"disabling worker decommission feature.")(decommissionSelf)
"disabling worker decommission feature.") {
self.send(WorkerSigPWRReceived)
true
}
} else {
logInfo("Worker decommissioning not enabled, SIGPWR will result in exiting.")
}
Expand Down Expand Up @@ -137,7 +140,8 @@ private[deploy] class Worker(
private var registered = false
private var connected = false
private var decommissioned = false
private val workerId = generateWorkerId()
// expose for test
private[spark] val workerId = generateWorkerId()
private val sparkHome =
if (sys.props.contains(IS_TESTING.key)) {
assert(sys.props.contains("spark.test.home"), "spark.test.home is not set!")
Expand Down Expand Up @@ -668,8 +672,14 @@ private[deploy] class Worker(
finishedApps += id
maybeCleanupApplication(id)

case WorkerDecommission(_, _) =>
case DecommissionWorker =>
decommissionSelf()

case WorkerSigPWRReceived =>
decommissionSelf()
// Tell the Master that we are starting decommissioning
// so it stops trying to launch executor/driver on us
sendToMaster(WorkerDecommissioning(workerId, self))
}

override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
Expand Down Expand Up @@ -768,16 +778,15 @@ private[deploy] class Worker(
}
}

private[deploy] def decommissionSelf(): Boolean = {
if (conf.get(config.DECOMMISSION_ENABLED)) {
logDebug("Decommissioning self")
private[deploy] def decommissionSelf(): Unit = {
if (conf.get(config.DECOMMISSION_ENABLED) && !decommissioned) {
decommissioned = true
sendToMaster(WorkerDecommission(workerId, self))
logInfo(s"Decommission worker $workerId.")
} else if (decommissioned) {
logWarning(s"Worker $workerId already started decommissioning.")
} else {
logWarning("Asked to decommission self, but decommissioning not enabled")
logWarning(s"Receive decommission request, but decommission feature is disabled.")
}
// Return true since can be called as a signal handler
true
}

private[worker] def handleDriverStateChanged(driverStateChanged: DriverStateChanged): Unit = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ import org.apache.spark.resource.ResourceProfile
import org.apache.spark.resource.ResourceProfile._
import org.apache.spark.resource.ResourceUtils._
import org.apache.spark.rpc._
import org.apache.spark.scheduler.{ExecutorDecommissionInfo, ExecutorLossReason, TaskDescription}
import org.apache.spark.scheduler.{ExecutorLossReason, TaskDescription}
import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._
import org.apache.spark.serializer.SerializerInstance
import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, SignalUtils, ThreadUtils, Utils}
Expand Down Expand Up @@ -79,12 +79,14 @@ private[spark] class CoarseGrainedExecutorBackend(
*/
private[executor] val taskResources = new mutable.HashMap[Long, Map[String, ResourceInformation]]

@volatile private var decommissioned = false
private var decommissioned = false

override def onStart(): Unit = {
logInfo("Registering PWR handler.")
SignalUtils.register("PWR", "Failed to register SIGPWR handler - " +
"disabling decommission feature.")(decommissionSelf)
if (env.conf.get(DECOMMISSION_ENABLED)) {
logInfo("Registering PWR handler to trigger decommissioning.")
SignalUtils.register("PWR", "Failed to register SIGPWR handler - " +
"disabling executor decommission feature.") (self.askSync[Boolean](ExecutorSigPWRReceived))
}

logInfo("Connecting to driver: " + driverUrl)
try {
Expand Down Expand Up @@ -166,17 +168,6 @@ private[spark] class CoarseGrainedExecutorBackend(
if (executor == null) {
exitExecutor(1, "Received LaunchTask command but executor was null")
} else {
if (decommissioned) {
val msg = "Asked to launch a task while decommissioned."
logError(msg)
driver match {
case Some(endpoint) =>
logInfo("Sending DecommissionExecutor to driver.")
endpoint.send(DecommissionExecutor(executorId, ExecutorDecommissionInfo(msg)))
case _ =>
logError("No registered driver to send Decommission to.")
}
}
val taskDesc = TaskDescription.decode(data.value)
logInfo("Got assigned task " + taskDesc.taskId)
taskResources(taskDesc.taskId) = taskDesc.resources
Expand Down Expand Up @@ -213,11 +204,31 @@ private[spark] class CoarseGrainedExecutorBackend(
logInfo(s"Received tokens of ${tokenBytes.length} bytes")
SparkHadoopUtil.get.addDelegationTokens(tokenBytes, env.conf)

case DecommissionSelf =>
logInfo("Received decommission self")
case DecommissionExecutor =>
decommissionSelf()
}

override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
case ExecutorSigPWRReceived =>
var driverNotified = false
try {
driver.foreach { driverRef =>
// Tell driver that we are starting decommissioning so it stops trying to schedule us
driverNotified = driverRef.askSync[Boolean](ExecutorDecommissioning(executorId))
if (driverNotified) decommissionSelf()
}
} catch {
case e: Exception =>
if (driverNotified) {
logError("Fail to decommission self (but driver has been notified).", e)
} else {
logError("Fail to tell driver that we are starting decommissioning", e)
}
decommissioned = false
}
context.reply(decommissioned)
}

override def onDisconnected(remoteAddress: RpcAddress): Unit = {
if (stopping.get()) {
logInfo(s"Driver from $remoteAddress disconnected during shutdown")
Expand Down Expand Up @@ -264,17 +275,20 @@ private[spark] class CoarseGrainedExecutorBackend(
System.exit(code)
}

private def decommissionSelf(): Boolean = {
val msg = "Decommissioning self w/sync"
private def decommissionSelf(): Unit = {
if (!env.conf.get(DECOMMISSION_ENABLED)) {
logWarning(s"Receive decommission request, but decommission feature is disabled.")
return
} else if (decommissioned) {
logWarning(s"Executor $executorId already started decommissioning.")
return
}
val msg = s"Decommission executor $executorId."
logInfo(msg)
try {
decommissioned = true
// Tell master we are are decommissioned so it stops trying to schedule us
if (driver.nonEmpty) {
driver.get.askSync[Boolean](DecommissionExecutor(
executorId, ExecutorDecommissionInfo(msg)))
} else {
logError("No driver to message decommissioning.")
if (env.conf.get(STORAGE_DECOMMISSION_ENABLED)) {
env.blockManager.decommissionBlockManager()
}
if (executor != null) {
executor.decommission()
Expand Down Expand Up @@ -333,12 +347,10 @@ private[spark] class CoarseGrainedExecutorBackend(
shutdownThread.start()

logInfo("Will exit when finished decommissioning")
// Return true since we are handling a signal
true
} catch {
case e: Exception =>
decommissioned = false
logError("Unexpected error while decommissioning self", e)
false
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -95,8 +95,17 @@ private[spark] object CoarseGrainedClusterMessages {
case class RemoveExecutor(executorId: String, reason: ExecutorLossReason)
extends CoarseGrainedClusterMessage

case class DecommissionExecutor(executorId: String, decommissionInfo: ExecutorDecommissionInfo)
extends CoarseGrainedClusterMessage
// A message that sent from executor to driver to tell driver that the executor has started
// decommissioning. It's used for the case where decommission is triggered at executor (e.g., K8S)
case class ExecutorDecommissioning(executorId: String) extends CoarseGrainedClusterMessage

// A message that sent from driver to executor to decommission that executor.
// It's used for Standalone's cases, where decommission is triggered at MasterWebUI or Worker.
object DecommissionExecutor extends CoarseGrainedClusterMessage

// A message that sent to the executor itself when it receives PWR signal,
// indicating the executor starts to decommission.
object ExecutorSigPWRReceived extends CoarseGrainedClusterMessage

case class RemoveWorker(workerId: String, host: String, message: String)
extends CoarseGrainedClusterMessage
Expand Down Expand Up @@ -136,7 +145,4 @@ private[spark] object CoarseGrainedClusterMessages {

// The message to check if `CoarseGrainedSchedulerBackend` thinks the executor is alive or not.
case class IsExecutorAlive(executorId: String) extends CoarseGrainedClusterMessage

// Used to ask an executor to decommission itself. (Can be an internal message)
case object DecommissionSelf extends CoarseGrainedClusterMessage
}
Loading

0 comments on commit edeecad

Please sign in to comment.