Skip to content

Commit edeecad

Browse files
Ngone51HyukjinKwon
authored andcommitted
[SPARK-32850][CORE][K8S] Simplify the RPC message flow of decommission
### 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 #29817 from Ngone51/simplify-decommission-rpc. Authored-by: yi.wu <yi.wu@databricks.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
1 parent 87b32f6 commit edeecad

File tree

19 files changed

+257
-166
lines changed

19 files changed

+257
-166
lines changed

core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala

Lines changed: 14 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -91,11 +91,13 @@ private[spark] trait ExecutorAllocationClient {
9191
* @param executorsAndDecomInfo identifiers of executors & decom info.
9292
* @param adjustTargetNumExecutors whether the target number of executors will be adjusted down
9393
* after these executors have been decommissioned.
94+
* @param triggeredByExecutor whether the decommission is triggered at executor.
9495
* @return the ids of the executors acknowledged by the cluster manager to be removed.
9596
*/
9697
def decommissionExecutors(
97-
executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
98-
adjustTargetNumExecutors: Boolean): Seq[String] = {
98+
executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
99+
adjustTargetNumExecutors: Boolean,
100+
triggeredByExecutor: Boolean): Seq[String] = {
99101
killExecutors(executorsAndDecomInfo.map(_._1),
100102
adjustTargetNumExecutors,
101103
countFailures = false)
@@ -109,14 +111,21 @@ private[spark] trait ExecutorAllocationClient {
109111
* @param executorId identifiers of executor to decommission
110112
* @param decommissionInfo information about the decommission (reason, host loss)
111113
* @param adjustTargetNumExecutors if we should adjust the target number of executors.
114+
* @param triggeredByExecutor whether the decommission is triggered at executor.
115+
* (TODO: add a new type like `ExecutorDecommissionInfo` for the
116+
* case where executor is decommissioned at executor first, so we
117+
* don't need this extra parameter.)
112118
* @return whether the request is acknowledged by the cluster manager.
113119
*/
114-
final def decommissionExecutor(executorId: String,
120+
final def decommissionExecutor(
121+
executorId: String,
115122
decommissionInfo: ExecutorDecommissionInfo,
116-
adjustTargetNumExecutors: Boolean): Boolean = {
123+
adjustTargetNumExecutors: Boolean,
124+
triggeredByExecutor: Boolean = false): Boolean = {
117125
val decommissionedExecutors = decommissionExecutors(
118126
Array((executorId, decommissionInfo)),
119-
adjustTargetNumExecutors = adjustTargetNumExecutors)
127+
adjustTargetNumExecutors = adjustTargetNumExecutors,
128+
triggeredByExecutor = triggeredByExecutor)
120129
decommissionedExecutors.nonEmpty && decommissionedExecutors(0).equals(executorId)
121130
}
122131

core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -580,7 +580,10 @@ private[spark] class ExecutorAllocationManager(
580580
if (decommissionEnabled) {
581581
val executorIdsWithoutHostLoss = executorIdsToBeRemoved.toSeq.map(
582582
id => (id, ExecutorDecommissionInfo("spark scale down"))).toArray
583-
client.decommissionExecutors(executorIdsWithoutHostLoss, adjustTargetNumExecutors = false)
583+
client.decommissionExecutors(
584+
executorIdsWithoutHostLoss,
585+
adjustTargetNumExecutors = false,
586+
triggeredByExecutor = false)
584587
} else {
585588
client.killExecutors(executorIdsToBeRemoved.toSeq, adjustTargetNumExecutors = false,
586589
countFailures = false, force = false)

core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala

Lines changed: 27 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -61,13 +61,35 @@ private[deploy] object DeployMessages {
6161
}
6262

6363
/**
64+
* An internal message that used by Master itself, in order to handle the
65+
* `DecommissionWorkersOnHosts` request from `MasterWebUI` asynchronously.
66+
* @param ids A collection of Worker ids, which should be decommissioned.
67+
*/
68+
case class DecommissionWorkers(ids: Seq[String]) extends DeployMessage
69+
70+
/**
71+
* A message that sent from Master to Worker to decommission the Worker.
72+
* It's used for the case where decommission is triggered at MasterWebUI.
73+
*
74+
* Note that decommission a Worker will cause all the executors on that Worker
75+
* to be decommissioned as well.
76+
*/
77+
object DecommissionWorker extends DeployMessage
78+
79+
/**
80+
* A message that sent by the Worker to itself when it receives PWR signal,
81+
* indicating the Worker starts to decommission.
82+
*/
83+
object WorkerSigPWRReceived extends DeployMessage
84+
85+
/**
86+
* A message sent from Worker to Master to tell Master that the Worker has started
87+
* decommissioning. It's used for the case where decommission is triggered at Worker.
88+
*
6489
* @param id the worker id
65-
* @param worker the worker endpoint ref
90+
* @param workerRef the worker endpoint ref
6691
*/
67-
case class WorkerDecommission(
68-
id: String,
69-
worker: RpcEndpointRef)
70-
extends DeployMessage
92+
case class WorkerDecommissioning(id: String, workerRef: RpcEndpointRef) extends DeployMessage
7193

7294
case class ExecutorStateChanged(
7395
appId: String,

core/src/main/scala/org/apache/spark/deploy/master/Master.scala

Lines changed: 15 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -245,15 +245,27 @@ private[deploy] class Master(
245245
logError("Leadership has been revoked -- master shutting down.")
246246
System.exit(0)
247247

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

256+
case DecommissionWorkers(ids) =>
257+
// The caller has already checked the state when handling DecommissionWorkersOnHosts,
258+
// so it should not be the STANDBY
259+
assert(state != RecoveryState.STANDBY)
260+
ids.foreach ( id =>
261+
// We use foreach since get gives us an option and we can skip the failures.
262+
idToWorker.get(id).foreach { w =>
263+
decommissionWorker(w)
264+
// Also send a message to the worker node to notify.
265+
w.endpoint.send(DecommissionWorker)
266+
}
267+
)
268+
257269
case RegisterWorker(
258270
id, workerHost, workerPort, workerRef, cores, memory, workerWebUiUrl,
259271
masterAddress, resources) =>
@@ -891,10 +903,7 @@ private[deploy] class Master(
891903
logInfo(s"Decommissioning the workers with host:ports ${workersToRemoveHostPorts}")
892904

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

899908
// Return the count of workers actually removed
900909
workersToRemove.size

core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala

Lines changed: 19 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -70,7 +70,10 @@ private[deploy] class Worker(
7070
if (conf.get(config.DECOMMISSION_ENABLED)) {
7171
logInfo("Registering SIGPWR handler to trigger decommissioning.")
7272
SignalUtils.register("PWR", "Failed to register SIGPWR handler - " +
73-
"disabling worker decommission feature.")(decommissionSelf)
73+
"disabling worker decommission feature.") {
74+
self.send(WorkerSigPWRReceived)
75+
true
76+
}
7477
} else {
7578
logInfo("Worker decommissioning not enabled, SIGPWR will result in exiting.")
7679
}
@@ -137,7 +140,8 @@ private[deploy] class Worker(
137140
private var registered = false
138141
private var connected = false
139142
private var decommissioned = false
140-
private val workerId = generateWorkerId()
143+
// expose for test
144+
private[spark] val workerId = generateWorkerId()
141145
private val sparkHome =
142146
if (sys.props.contains(IS_TESTING.key)) {
143147
assert(sys.props.contains("spark.test.home"), "spark.test.home is not set!")
@@ -668,8 +672,14 @@ private[deploy] class Worker(
668672
finishedApps += id
669673
maybeCleanupApplication(id)
670674

671-
case WorkerDecommission(_, _) =>
675+
case DecommissionWorker =>
676+
decommissionSelf()
677+
678+
case WorkerSigPWRReceived =>
672679
decommissionSelf()
680+
// Tell the Master that we are starting decommissioning
681+
// so it stops trying to launch executor/driver on us
682+
sendToMaster(WorkerDecommissioning(workerId, self))
673683
}
674684

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

771-
private[deploy] def decommissionSelf(): Boolean = {
772-
if (conf.get(config.DECOMMISSION_ENABLED)) {
773-
logDebug("Decommissioning self")
781+
private[deploy] def decommissionSelf(): Unit = {
782+
if (conf.get(config.DECOMMISSION_ENABLED) && !decommissioned) {
774783
decommissioned = true
775-
sendToMaster(WorkerDecommission(workerId, self))
784+
logInfo(s"Decommission worker $workerId.")
785+
} else if (decommissioned) {
786+
logWarning(s"Worker $workerId already started decommissioning.")
776787
} else {
777-
logWarning("Asked to decommission self, but decommissioning not enabled")
788+
logWarning(s"Receive decommission request, but decommission feature is disabled.")
778789
}
779-
// Return true since can be called as a signal handler
780-
true
781790
}
782791

783792
private[worker] def handleDriverStateChanged(driverStateChanged: DriverStateChanged): Unit = {

core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala

Lines changed: 41 additions & 29 deletions
Original file line numberDiff line numberDiff line change
@@ -40,7 +40,7 @@ import org.apache.spark.resource.ResourceProfile
4040
import org.apache.spark.resource.ResourceProfile._
4141
import org.apache.spark.resource.ResourceUtils._
4242
import org.apache.spark.rpc._
43-
import org.apache.spark.scheduler.{ExecutorDecommissionInfo, ExecutorLossReason, TaskDescription}
43+
import org.apache.spark.scheduler.{ExecutorLossReason, TaskDescription}
4444
import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._
4545
import org.apache.spark.serializer.SerializerInstance
4646
import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, SignalUtils, ThreadUtils, Utils}
@@ -79,12 +79,14 @@ private[spark] class CoarseGrainedExecutorBackend(
7979
*/
8080
private[executor] val taskResources = new mutable.HashMap[Long, Map[String, ResourceInformation]]
8181

82-
@volatile private var decommissioned = false
82+
private var decommissioned = false
8383

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

8991
logInfo("Connecting to driver: " + driverUrl)
9092
try {
@@ -166,17 +168,6 @@ private[spark] class CoarseGrainedExecutorBackend(
166168
if (executor == null) {
167169
exitExecutor(1, "Received LaunchTask command but executor was null")
168170
} else {
169-
if (decommissioned) {
170-
val msg = "Asked to launch a task while decommissioned."
171-
logError(msg)
172-
driver match {
173-
case Some(endpoint) =>
174-
logInfo("Sending DecommissionExecutor to driver.")
175-
endpoint.send(DecommissionExecutor(executorId, ExecutorDecommissionInfo(msg)))
176-
case _ =>
177-
logError("No registered driver to send Decommission to.")
178-
}
179-
}
180171
val taskDesc = TaskDescription.decode(data.value)
181172
logInfo("Got assigned task " + taskDesc.taskId)
182173
taskResources(taskDesc.taskId) = taskDesc.resources
@@ -213,11 +204,31 @@ private[spark] class CoarseGrainedExecutorBackend(
213204
logInfo(s"Received tokens of ${tokenBytes.length} bytes")
214205
SparkHadoopUtil.get.addDelegationTokens(tokenBytes, env.conf)
215206

216-
case DecommissionSelf =>
217-
logInfo("Received decommission self")
207+
case DecommissionExecutor =>
218208
decommissionSelf()
219209
}
220210

211+
override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
212+
case ExecutorSigPWRReceived =>
213+
var driverNotified = false
214+
try {
215+
driver.foreach { driverRef =>
216+
// Tell driver that we are starting decommissioning so it stops trying to schedule us
217+
driverNotified = driverRef.askSync[Boolean](ExecutorDecommissioning(executorId))
218+
if (driverNotified) decommissionSelf()
219+
}
220+
} catch {
221+
case e: Exception =>
222+
if (driverNotified) {
223+
logError("Fail to decommission self (but driver has been notified).", e)
224+
} else {
225+
logError("Fail to tell driver that we are starting decommissioning", e)
226+
}
227+
decommissioned = false
228+
}
229+
context.reply(decommissioned)
230+
}
231+
221232
override def onDisconnected(remoteAddress: RpcAddress): Unit = {
222233
if (stopping.get()) {
223234
logInfo(s"Driver from $remoteAddress disconnected during shutdown")
@@ -264,17 +275,20 @@ private[spark] class CoarseGrainedExecutorBackend(
264275
System.exit(code)
265276
}
266277

267-
private def decommissionSelf(): Boolean = {
268-
val msg = "Decommissioning self w/sync"
278+
private def decommissionSelf(): Unit = {
279+
if (!env.conf.get(DECOMMISSION_ENABLED)) {
280+
logWarning(s"Receive decommission request, but decommission feature is disabled.")
281+
return
282+
} else if (decommissioned) {
283+
logWarning(s"Executor $executorId already started decommissioning.")
284+
return
285+
}
286+
val msg = s"Decommission executor $executorId."
269287
logInfo(msg)
270288
try {
271289
decommissioned = true
272-
// Tell master we are are decommissioned so it stops trying to schedule us
273-
if (driver.nonEmpty) {
274-
driver.get.askSync[Boolean](DecommissionExecutor(
275-
executorId, ExecutorDecommissionInfo(msg)))
276-
} else {
277-
logError("No driver to message decommissioning.")
290+
if (env.conf.get(STORAGE_DECOMMISSION_ENABLED)) {
291+
env.blockManager.decommissionBlockManager()
278292
}
279293
if (executor != null) {
280294
executor.decommission()
@@ -333,12 +347,10 @@ private[spark] class CoarseGrainedExecutorBackend(
333347
shutdownThread.start()
334348

335349
logInfo("Will exit when finished decommissioning")
336-
// Return true since we are handling a signal
337-
true
338350
} catch {
339351
case e: Exception =>
352+
decommissioned = false
340353
logError("Unexpected error while decommissioning self", e)
341-
false
342354
}
343355
}
344356
}

core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala

Lines changed: 11 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -95,8 +95,17 @@ private[spark] object CoarseGrainedClusterMessages {
9595
case class RemoveExecutor(executorId: String, reason: ExecutorLossReason)
9696
extends CoarseGrainedClusterMessage
9797

98-
case class DecommissionExecutor(executorId: String, decommissionInfo: ExecutorDecommissionInfo)
99-
extends CoarseGrainedClusterMessage
98+
// A message that sent from executor to driver to tell driver that the executor has started
99+
// decommissioning. It's used for the case where decommission is triggered at executor (e.g., K8S)
100+
case class ExecutorDecommissioning(executorId: String) extends CoarseGrainedClusterMessage
101+
102+
// A message that sent from driver to executor to decommission that executor.
103+
// It's used for Standalone's cases, where decommission is triggered at MasterWebUI or Worker.
104+
object DecommissionExecutor extends CoarseGrainedClusterMessage
105+
106+
// A message that sent to the executor itself when it receives PWR signal,
107+
// indicating the executor starts to decommission.
108+
object ExecutorSigPWRReceived extends CoarseGrainedClusterMessage
100109

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

137146
// The message to check if `CoarseGrainedSchedulerBackend` thinks the executor is alive or not.
138147
case class IsExecutorAlive(executorId: String) extends CoarseGrainedClusterMessage
139-
140-
// Used to ask an executor to decommission itself. (Can be an internal message)
141-
case object DecommissionSelf extends CoarseGrainedClusterMessage
142148
}

0 commit comments

Comments
 (0)