You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2020/09/15 08:57:29 UTC

[GitHub] [spark] Ngone51 opened a new pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Ngone51 opened a new pull request #29722:
URL: https://github.com/apache/spark/pull/29722


   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
     2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
     4. Be sure to keep the PR description updated to reflect all changes.
     5. Please write your PR title to summarize what this PR proposes.
     6. If possible, provide a concise example to reproduce the issue for a faster review.
     7. If you want to add a new configuration, please read the guideline first for naming configurations in
        'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
   -->
   
   ### What changes were proposed in this pull request?
   <!--
   Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   
   
   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?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   
   Before:
   
   <img width="1948" alt="WeChat56c00cc34d9785a67a544dca036d49da" src="https://user-images.githubusercontent.com/16397174/92850308-dc461c80-f41e-11ea-8ac0-287825f4e0c4.png">
   
   After:
   <img width="1958" alt="WeChatbbc878678dafa81b76fcd4fbec43a1c0" src="https://user-images.githubusercontent.com/16397174/92890294-9488ba80-f449-11ea-942c-e6436d6a61a4.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?
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   
   No.
   
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   -->
   
   Updated existing tests.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692455812


   **[Test build #128693 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128693/testReport)** for PR 29722 at commit [`f64063e`](https://github.com/apache/spark/commit/f64063ed7f3ac2851ddfbdf92b4961fbd1a192c6).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692670128


   **[Test build #128711 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128711/testReport)** for PR 29722 at commit [`9debec8`](https://github.com/apache/spark/commit/9debec82fb836ab334d93b18b36b4b7f2f9129cc).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692672223






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r487658824



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -1809,7 +1811,7 @@ private[spark] class BlockManager(
     blocksToRemove.size
   }
 
-  def decommissionBlockManager(): Unit = synchronized {
+  private[spark] def decommissionSelf(): Unit = synchronized {

Review comment:
       It conflicts with the newly added "decommissionBlockManager()".




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-693497782


   thanks all!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692513189


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/128693/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] agrawaldevesh commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
agrawaldevesh commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r488069363



##########
File path: core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
##########
@@ -79,12 +79,23 @@ 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.send(DecommissionExecutor)

Review comment:
       I think being async here is fine, but I would like to better understand the need for inlining the logic (removed from L273 below in here).
   
   Is ExecutorDecommissioning only supposed to be called when the executor receives a SIGPWR and NOT when it it is asked to decom by the driver (or in the standalone scenario) ? 
   
   Earlier it seemed that both the PWR signal handling and the other codepaths went through the same `decommissionSelf` code flow but now it they appear a bit different.
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r487692255



##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala
##########
@@ -95,8 +95,13 @@ 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 been
+  // used. It's used for the case where decommission is triggered at executor (e.g., K8S)

Review comment:
       `has been used` -> `has been decommissioned`

##########
File path: core/src/main/scala/org/apache/spark/deploy/master/Master.scala
##########
@@ -245,15 +245,19 @@ 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 WorkerDecommissioned(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.

Review comment:
       shall we keep this comment?

##########
File path: core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
##########
@@ -109,14 +110,22 @@ 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 by sending the
+   *                                 `DecommissionExecutor` from driver to executor

Review comment:
       then the name should be `triggeredByDriver`?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r487867415



##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +467,42 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
-        // Only bother decommissioning executors which are alive.
-        if (isExecutorActive(executorId)) {
-          executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
-        } else {
-          false
-        }
+      adjustTargetNumExecutors: Boolean,
+      triggeredByExecutor: Boolean): Seq[String] = withLock {
+    val executorsToDecommission = executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
+      // Only bother decommissioning executors which are alive.
+      if (isExecutorActive(executorId)) {
+        scheduler.executorDecommission(executorId, decomInfo)
+        executorsPendingDecommission(executorId) = decomInfo.workerHost
+        Some(executorId)
+      } else {
+        None
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
-
-  private def doDecommission(executorId: String,
-      decomInfo: ExecutorDecommissionInfo): Boolean = {
-
-    logInfo(s"Asking executor $executorId to decommissioning.")
-    scheduler.executorDecommission(executorId, decomInfo)
-    // Send decommission message to the executor (it could have originated on the executor
-    // but not necessarily).
-    CoarseGrainedSchedulerBackend.this.synchronized {
-      executorDataMap.get(executorId) match {
-        case Some(executorInfo) =>
-          executorInfo.executorEndpoint.send(DecommissionSelf)
-        case None =>
-          // Ignoring the executor since it is not registered.
-          logWarning(s"Attempted to decommission unknown executor $executorId.")
-          return false
-      }
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
+      scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)
     }
-    logInfo(s"Asked executor $executorId to decommission.")
 
-    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
-      try {
-        logInfo(s"Asking block manager corresponding to executor $executorId to decommission.")
-        scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))
-      } catch {
-        case e: Exception =>
-          logError("Unexpected error during block manager " +
-            s"decommissioning for executor $executorId: ${e.toString}", e)
-          return false
+    if (triggeredByExecutor) {
+      executorsToDecommission.foreach { executorId =>
+        logInfo(s"Asking executor $executorId to decommissioning.")
+        executorDataMap(executorId).executorEndpoint.send(DecommissionExecutor)

Review comment:
       We don't need it now. Because the whole block is under the same lock protection(`withLock`). And since we've already ensured `isExecutorActive` above, `executorDataMap` must contains the `executorId` at this point.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692068264


   **[Test build #128654 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128654/testReport)** for PR 29722 at commit [`86b8b74`](https://github.com/apache/spark/commit/86b8b741b87d44d2da58590467e2cdedf0e4bf4b).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-691898896






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692049323


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-691991559


   **[Test build #128633 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128633/testReport)** for PR 29722 at commit [`c922fb0`](https://github.com/apache/spark/commit/c922fb0421be63a23469745d187ee6c9b1755156).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] agrawaldevesh commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
agrawaldevesh commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r487349021



##########
File path: core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
##########
@@ -94,8 +94,9 @@ private[spark] trait ExecutorAllocationClient {
    * @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)],

Review comment:
       I keep messing this up myself, but is this indentation style change intentional ? If 2 spaces are okay then can we remain at that ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
##########
@@ -61,13 +61,28 @@ 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 are pending to be decommissioned.

Review comment:
       nit: which are pending to be decommissioned -> which should be decommissioned ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/master/Master.scala
##########
@@ -245,14 +245,14 @@ 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))

Review comment:
       We don't need to handle the MasterInStandby case anymore ? I guess all messages from worker to master should have this check, right ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
##########
@@ -768,11 +768,13 @@ private[deploy] class Worker(
     }
   }
 
-  private[deploy] def decommissionSelf(): Boolean = {
+  private[deploy] def decommissionSelf(fromMaster: Boolean): Boolean = {
     if (conf.get(config.DECOMMISSION_ENABLED)) {
       logDebug("Decommissioning self")
       decommissioned = true
-      sendToMaster(WorkerDecommission(workerId, self))
+      if (!fromMaster) {
+        sendToMaster(WorkerDecommissioned(workerId))

Review comment:
       I wonder if it is trite to add a comment like: "No need to notify the master if the decommission message already came from it"

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala
##########
@@ -95,8 +95,13 @@ 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 been
+  // used. It's used for the case where decommission is triggered at executor (e.g., K8S)
+  case class ExecutorDecommissioned(executorId: String)
+
+  // A message that sent from driver to executor to decommission that executor.
+  // It's used for Standalone's case yet, where decommission is triggered at Worker.

Review comment:
       remove 'yet'. Are you sure that this is when the decommission is triggered at the Worker ? I think it can also be used when the decommissioning is triggered at the MasterWebUI (not necessarily a SIGPWR at the Worker).

##########
File path: core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
##########
@@ -166,17 +166,6 @@ private[spark] class CoarseGrainedExecutorBackend(
       if (executor == null) {
         exitExecutor(1, "Received LaunchTask command but executor was null")
       } else {
-        if (decommissioned) {

Review comment:
       Are you sure that this case won't happen ? Or perhaps you mean that even if it does happen, it only lasts for a short duration: (until the Driver gets to know that the executor is decommissioned), that its not worth handling it ? If so, I agree with you. 
   
   But just want to confirm when can this case even happen: task launched on a decommissioned executor ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/master/Master.scala
##########
@@ -891,16 +891,13 @@ 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
   }
 
-  private def decommissionWorker(worker: WorkerInfo): Unit = {
+  private def decommissionWorker(worker: WorkerInfo, sentFromWorker: Boolean): Unit = {

Review comment:
       Would triggeredByWorker be a better name than sentFromWorker ?

##########
File path: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
##########
@@ -581,7 +581,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,
+          decommissionFromDriver = true)

Review comment:
       I think decommissionFromDriver = false means that decommission is triggered by a SIGPWR on the executor. 
   
   Whereas, decommissionFromDriver = true means that it might be triggered form either the Master or the Dynamic allocation manager. 
   
   If so, should we flip decommissionFromDriver and just call it decomTriggeredByExecutor ? 

##########
File path: core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
##########
@@ -768,11 +768,13 @@ private[deploy] class Worker(
     }
   }
 
-  private[deploy] def decommissionSelf(): Boolean = {
+  private[deploy] def decommissionSelf(fromMaster: Boolean): Boolean = {

Review comment:
       For consistency: triggeredByMaster instead of fromMaster ?

##########
File path: core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
##########
@@ -264,17 +253,21 @@ private[spark] class CoarseGrainedExecutorBackend(
     System.exit(code)
   }
 
-  private def decommissionSelf(): Boolean = {
+  private def decommissionSelf(fromDriver: Boolean): Boolean = {
     val msg = "Decommissioning self w/sync"
     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)) {

Review comment:
       I see, so is this the now ONLY place where block manager decommissioning is triggered ? Good.

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -272,10 +268,16 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
         removeWorker(workerId, host, message)
         context.reply(true)
 
-      case DecommissionExecutor(executorId, decommissionInfo) =>
-        logError(s"Received decommission executor message ${executorId}: ${decommissionInfo}.")
-        context.reply(decommissionExecutor(executorId, decommissionInfo,
-          adjustTargetNumExecutors = false))
+      case ExecutorDecommissioned(executorId) =>
+        logWarning(s"Received executor $executorId decommissioned message")
+        context.reply(
+          decommissionExecutor(
+            executorId,
+            ExecutorDecommissionInfo(s"Executor $executorId is decommissioned."),
+            adjustTargetNumExecutors = false,
+            // TODO: add a new type like `ExecutorDecommissionInfo` for the case where executor

Review comment:
       This is a good TODO comment, should we add this at the definition of the `decommissionExecutor` method instead of at the caller ?

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {

Review comment:
       nice ! I like the change to flatMap and withLock.

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -1809,7 +1811,7 @@ private[spark] class BlockManager(
     blocksToRemove.size
   }
 
-  def decommissionBlockManager(): Unit = synchronized {
+  private[spark] def decommissionSelf(): Unit = synchronized {

Review comment:
       Didn't follow the need for the name change.

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)
 
-  private def doDecommission(executorId: String,
-      decomInfo: ExecutorDecommissionInfo): Boolean = {
-
-    logInfo(s"Asking executor $executorId to decommissioning.")
-    scheduler.executorDecommission(executorId, decomInfo)
-    // Send decommission message to the executor (it could have originated on the executor
-    // but not necessarily).
-    CoarseGrainedSchedulerBackend.this.synchronized {
-      executorDataMap.get(executorId) match {
-        case Some(executorInfo) =>
-          executorInfo.executorEndpoint.send(DecommissionSelf)
-        case None =>
-          // Ignoring the executor since it is not registered.
-          logWarning(s"Attempted to decommission unknown executor $executorId.")
-          return false
-      }
-    }
-    logInfo(s"Asked executor $executorId to decommission.")
-
-    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
-      try {
-        logInfo(s"Asking block manager corresponding to executor $executorId to decommission.")
-        scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))
-      } catch {
-        case e: Exception =>
-          logError("Unexpected error during block manager " +
-            s"decommissioning for executor $executorId: ${e.toString}", e)
-          return false
+    if (decommissionFromDriver) {
+      CoarseGrainedSchedulerBackend.this.synchronized {

Review comment:
       Why the lock here ? It wasn't locked before (I think).

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)
 
-  private def doDecommission(executorId: String,
-      decomInfo: ExecutorDecommissionInfo): Boolean = {
-
-    logInfo(s"Asking executor $executorId to decommissioning.")
-    scheduler.executorDecommission(executorId, decomInfo)
-    // Send decommission message to the executor (it could have originated on the executor
-    // but not necessarily).
-    CoarseGrainedSchedulerBackend.this.synchronized {
-      executorDataMap.get(executorId) match {
-        case Some(executorInfo) =>
-          executorInfo.executorEndpoint.send(DecommissionSelf)
-        case None =>
-          // Ignoring the executor since it is not registered.
-          logWarning(s"Attempted to decommission unknown executor $executorId.")
-          return false
-      }
-    }
-    logInfo(s"Asked executor $executorId to decommission.")
-
-    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
-      try {
-        logInfo(s"Asking block manager corresponding to executor $executorId to decommission.")
-        scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))
-      } catch {
-        case e: Exception =>
-          logError("Unexpected error during block manager " +
-            s"decommissioning for executor $executorId: ${e.toString}", e)
-          return false
+    if (decommissionFromDriver) {
+      CoarseGrainedSchedulerBackend.this.synchronized {
+        executorsToDecommission.foreach { executorId =>
+          logInfo(s"Asking executor $executorId to decommissioning.")
+          executorDataMap(executorId).executorEndpoint.send(DecommissionExecutor)

Review comment:
        executorDataMap(executorId) might be None: if an executor was removed b/w the end of withLock on L485 to L500. The original code handled that. Should we handle it to ? (We don't have to log anything, just not crash the driver).

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)

Review comment:
       Should this check be guarded by `conf.get(STORAGE_DECOMMISSION_ENABLED)` ? Also should this only be done in case of decommissionFromDriver == true ?
   

##########
File path: core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala
##########
@@ -31,7 +31,7 @@ import org.apache.spark.util.{RpcUtils, SerializableBuffer, ThreadUtils}
 class WorkerDecommissionSuite extends SparkFunSuite with LocalSparkContext {
 
   override def beforeEach(): Unit = {
-    val conf = new SparkConf().setAppName("test").setMaster("local")

Review comment:
       Is master = "local" the default ? And that's why the .setMaster("local") is removed ?

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)
 
-  private def doDecommission(executorId: String,
-      decomInfo: ExecutorDecommissionInfo): Boolean = {
-
-    logInfo(s"Asking executor $executorId to decommissioning.")
-    scheduler.executorDecommission(executorId, decomInfo)
-    // Send decommission message to the executor (it could have originated on the executor
-    // but not necessarily).
-    CoarseGrainedSchedulerBackend.this.synchronized {
-      executorDataMap.get(executorId) match {
-        case Some(executorInfo) =>
-          executorInfo.executorEndpoint.send(DecommissionSelf)
-        case None =>
-          // Ignoring the executor since it is not registered.
-          logWarning(s"Attempted to decommission unknown executor $executorId.")
-          return false
-      }
-    }
-    logInfo(s"Asked executor $executorId to decommission.")
-
-    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
-      try {
-        logInfo(s"Asking block manager corresponding to executor $executorId to decommission.")
-        scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))

Review comment:
       One of the differences with the old design was that the 'Clearing of the block managers from the block manager master and the actual message to the decommission block manager' happened in one place (in scheduler.sc.env.blockManager.master.decommissionBlockManagers). 
   
   Whereas now these two things are split up: The scheduler.sc.env.blockManager.master.decommissionBlockManagers no longer sends the message, just updates its in memory state.
   
   Is there a problem with this difference ? Is some race opened up or closed ?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r487673792



##########
File path: core/src/main/scala/org/apache/spark/deploy/master/Master.scala
##########
@@ -891,16 +891,13 @@ 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
   }
 
-  private def decommissionWorker(worker: WorkerInfo): Unit = {
+  private def decommissionWorker(worker: WorkerInfo, sentFromWorker: Boolean): Unit = {

Review comment:
       Sounds good to me!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 closed pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
Ngone51 closed pull request #29722:
URL: https://github.com/apache/spark/pull/29722


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-691861748


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on pull request #29722: [SPARK-32850][CORE] Simply the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-690851376


   @cloud-fan @agrawaldevesh @holdenk Please take a look, thanks!
    


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #29722: [SPARK-32850][CORE] Simply the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-690851353






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] agrawaldevesh commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
agrawaldevesh commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r487349021



##########
File path: core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
##########
@@ -94,8 +94,9 @@ private[spark] trait ExecutorAllocationClient {
    * @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)],

Review comment:
       I keep messing this up myself, but is this indentation style change intentional ? If 2 spaces are okay then can we remain at that ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
##########
@@ -61,13 +61,28 @@ 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 are pending to be decommissioned.

Review comment:
       nit: which are pending to be decommissioned -> which should be decommissioned ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/master/Master.scala
##########
@@ -245,14 +245,14 @@ 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))

Review comment:
       We don't need to handle the MasterInStandby case anymore ? I guess all messages from worker to master should have this check, right ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
##########
@@ -768,11 +768,13 @@ private[deploy] class Worker(
     }
   }
 
-  private[deploy] def decommissionSelf(): Boolean = {
+  private[deploy] def decommissionSelf(fromMaster: Boolean): Boolean = {
     if (conf.get(config.DECOMMISSION_ENABLED)) {
       logDebug("Decommissioning self")
       decommissioned = true
-      sendToMaster(WorkerDecommission(workerId, self))
+      if (!fromMaster) {
+        sendToMaster(WorkerDecommissioned(workerId))

Review comment:
       I wonder if it is trite to add a comment like: "No need to notify the master if the decommission message already came from it"

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala
##########
@@ -95,8 +95,13 @@ 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 been
+  // used. It's used for the case where decommission is triggered at executor (e.g., K8S)
+  case class ExecutorDecommissioned(executorId: String)
+
+  // A message that sent from driver to executor to decommission that executor.
+  // It's used for Standalone's case yet, where decommission is triggered at Worker.

Review comment:
       remove 'yet'. Are you sure that this is when the decommission is triggered at the Worker ? I think it can also be used when the decommissioning is triggered at the MasterWebUI (not necessarily a SIGPWR at the Worker).

##########
File path: core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
##########
@@ -166,17 +166,6 @@ private[spark] class CoarseGrainedExecutorBackend(
       if (executor == null) {
         exitExecutor(1, "Received LaunchTask command but executor was null")
       } else {
-        if (decommissioned) {

Review comment:
       Are you sure that this case won't happen ? Or perhaps you mean that even if it does happen, it only lasts for a short duration: (until the Driver gets to know that the executor is decommissioned), that its not worth handling it ? If so, I agree with you. 
   
   But just want to confirm when can this case even happen: task launched on a decommissioned executor ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/master/Master.scala
##########
@@ -891,16 +891,13 @@ 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
   }
 
-  private def decommissionWorker(worker: WorkerInfo): Unit = {
+  private def decommissionWorker(worker: WorkerInfo, sentFromWorker: Boolean): Unit = {

Review comment:
       Would triggeredByWorker be a better name than sentFromWorker ?

##########
File path: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
##########
@@ -581,7 +581,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,
+          decommissionFromDriver = true)

Review comment:
       I think decommissionFromDriver = false means that decommission is triggered by a SIGPWR on the executor. 
   
   Whereas, decommissionFromDriver = true means that it might be triggered form either the Master or the Dynamic allocation manager. 
   
   If so, should we flip decommissionFromDriver and just call it decomTriggeredByExecutor ? 

##########
File path: core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
##########
@@ -768,11 +768,13 @@ private[deploy] class Worker(
     }
   }
 
-  private[deploy] def decommissionSelf(): Boolean = {
+  private[deploy] def decommissionSelf(fromMaster: Boolean): Boolean = {

Review comment:
       For consistency: triggeredByMaster instead of fromMaster ?

##########
File path: core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
##########
@@ -264,17 +253,21 @@ private[spark] class CoarseGrainedExecutorBackend(
     System.exit(code)
   }
 
-  private def decommissionSelf(): Boolean = {
+  private def decommissionSelf(fromDriver: Boolean): Boolean = {
     val msg = "Decommissioning self w/sync"
     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)) {

Review comment:
       I see, so is this the now ONLY place where block manager decommissioning is triggered ? Good.

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -272,10 +268,16 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
         removeWorker(workerId, host, message)
         context.reply(true)
 
-      case DecommissionExecutor(executorId, decommissionInfo) =>
-        logError(s"Received decommission executor message ${executorId}: ${decommissionInfo}.")
-        context.reply(decommissionExecutor(executorId, decommissionInfo,
-          adjustTargetNumExecutors = false))
+      case ExecutorDecommissioned(executorId) =>
+        logWarning(s"Received executor $executorId decommissioned message")
+        context.reply(
+          decommissionExecutor(
+            executorId,
+            ExecutorDecommissionInfo(s"Executor $executorId is decommissioned."),
+            adjustTargetNumExecutors = false,
+            // TODO: add a new type like `ExecutorDecommissionInfo` for the case where executor

Review comment:
       This is a good TODO comment, should we add this at the definition of the `decommissionExecutor` method instead of at the caller ?

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {

Review comment:
       nice ! I like the change to flatMap and withLock.

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -1809,7 +1811,7 @@ private[spark] class BlockManager(
     blocksToRemove.size
   }
 
-  def decommissionBlockManager(): Unit = synchronized {
+  private[spark] def decommissionSelf(): Unit = synchronized {

Review comment:
       Didn't follow the need for the name change.

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)
 
-  private def doDecommission(executorId: String,
-      decomInfo: ExecutorDecommissionInfo): Boolean = {
-
-    logInfo(s"Asking executor $executorId to decommissioning.")
-    scheduler.executorDecommission(executorId, decomInfo)
-    // Send decommission message to the executor (it could have originated on the executor
-    // but not necessarily).
-    CoarseGrainedSchedulerBackend.this.synchronized {
-      executorDataMap.get(executorId) match {
-        case Some(executorInfo) =>
-          executorInfo.executorEndpoint.send(DecommissionSelf)
-        case None =>
-          // Ignoring the executor since it is not registered.
-          logWarning(s"Attempted to decommission unknown executor $executorId.")
-          return false
-      }
-    }
-    logInfo(s"Asked executor $executorId to decommission.")
-
-    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
-      try {
-        logInfo(s"Asking block manager corresponding to executor $executorId to decommission.")
-        scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))
-      } catch {
-        case e: Exception =>
-          logError("Unexpected error during block manager " +
-            s"decommissioning for executor $executorId: ${e.toString}", e)
-          return false
+    if (decommissionFromDriver) {
+      CoarseGrainedSchedulerBackend.this.synchronized {

Review comment:
       Why the lock here ? It wasn't locked before (I think).

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)
 
-  private def doDecommission(executorId: String,
-      decomInfo: ExecutorDecommissionInfo): Boolean = {
-
-    logInfo(s"Asking executor $executorId to decommissioning.")
-    scheduler.executorDecommission(executorId, decomInfo)
-    // Send decommission message to the executor (it could have originated on the executor
-    // but not necessarily).
-    CoarseGrainedSchedulerBackend.this.synchronized {
-      executorDataMap.get(executorId) match {
-        case Some(executorInfo) =>
-          executorInfo.executorEndpoint.send(DecommissionSelf)
-        case None =>
-          // Ignoring the executor since it is not registered.
-          logWarning(s"Attempted to decommission unknown executor $executorId.")
-          return false
-      }
-    }
-    logInfo(s"Asked executor $executorId to decommission.")
-
-    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
-      try {
-        logInfo(s"Asking block manager corresponding to executor $executorId to decommission.")
-        scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))
-      } catch {
-        case e: Exception =>
-          logError("Unexpected error during block manager " +
-            s"decommissioning for executor $executorId: ${e.toString}", e)
-          return false
+    if (decommissionFromDriver) {
+      CoarseGrainedSchedulerBackend.this.synchronized {
+        executorsToDecommission.foreach { executorId =>
+          logInfo(s"Asking executor $executorId to decommissioning.")
+          executorDataMap(executorId).executorEndpoint.send(DecommissionExecutor)

Review comment:
        executorDataMap(executorId) might be None: if an executor was removed b/w the end of withLock on L485 to L500. The original code handled that. Should we handle it to ? (We don't have to log anything, just not crash the driver).

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)

Review comment:
       Should this check be guarded by `conf.get(STORAGE_DECOMMISSION_ENABLED)` ? Also should this only be done in case of decommissionFromDriver == true ?
   

##########
File path: core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala
##########
@@ -31,7 +31,7 @@ import org.apache.spark.util.{RpcUtils, SerializableBuffer, ThreadUtils}
 class WorkerDecommissionSuite extends SparkFunSuite with LocalSparkContext {
 
   override def beforeEach(): Unit = {
-    val conf = new SparkConf().setAppName("test").setMaster("local")

Review comment:
       Is master = "local" the default ? And that's why the .setMaster("local") is removed ?

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)
 
-  private def doDecommission(executorId: String,
-      decomInfo: ExecutorDecommissionInfo): Boolean = {
-
-    logInfo(s"Asking executor $executorId to decommissioning.")
-    scheduler.executorDecommission(executorId, decomInfo)
-    // Send decommission message to the executor (it could have originated on the executor
-    // but not necessarily).
-    CoarseGrainedSchedulerBackend.this.synchronized {
-      executorDataMap.get(executorId) match {
-        case Some(executorInfo) =>
-          executorInfo.executorEndpoint.send(DecommissionSelf)
-        case None =>
-          // Ignoring the executor since it is not registered.
-          logWarning(s"Attempted to decommission unknown executor $executorId.")
-          return false
-      }
-    }
-    logInfo(s"Asked executor $executorId to decommission.")
-
-    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
-      try {
-        logInfo(s"Asking block manager corresponding to executor $executorId to decommission.")
-        scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))

Review comment:
       One of the differences with the old design was that the 'Clearing of the block managers from the block manager master and the actual message to the decommission block manager' happened in one place (in scheduler.sc.env.blockManager.master.decommissionBlockManagers). 
   
   Whereas now these two things are split up: The scheduler.sc.env.blockManager.master.decommissionBlockManagers no longer sends the message, just updates its in memory state.
   
   Is there a problem with this difference ? Is some race opened up or closed ?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692670633






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692670128


   **[Test build #128711 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128711/testReport)** for PR 29722 at commit [`9debec8`](https://github.com/apache/spark/commit/9debec82fb836ab334d93b18b36b4b7f2f9129cc).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-691898896






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r487657421



##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala
##########
@@ -95,8 +95,13 @@ 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 been
+  // used. It's used for the case where decommission is triggered at executor (e.g., K8S)
+  case class ExecutorDecommissioned(executorId: String)
+
+  // A message that sent from driver to executor to decommission that executor.
+  // It's used for Standalone's case yet, where decommission is triggered at Worker.

Review comment:
       Ok, I actually mean it used for both two cases as they would trigger decommission at Worker side anyway. I'll update the comment to make it clearer. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-691861758


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/128620/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-693213903


   **[Test build #128748 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128748/testReport)** for PR 29722 at commit [`290d2c0`](https://github.com/apache/spark/commit/290d2c08a1adfce3c0c4afe1cb8c9e214b25ea3d).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692066878


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/128653/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29722: [SPARK-32850][CORE] Simply the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-690851701






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692066868






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r488056433



##########
File path: core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
##########
@@ -79,12 +79,23 @@ 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.send(DecommissionExecutor)

Review comment:
       SGTM




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simply the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-690923986






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-693216769


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/128748/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-691848628






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] agrawaldevesh commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
agrawaldevesh commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r487349021



##########
File path: core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
##########
@@ -94,8 +94,9 @@ private[spark] trait ExecutorAllocationClient {
    * @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)],

Review comment:
       I keep messing this up myself, but is this indentation style change intentional ? If 2 spaces are okay then can we remain at that ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
##########
@@ -61,13 +61,28 @@ 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 are pending to be decommissioned.

Review comment:
       nit: which are pending to be decommissioned -> which should be decommissioned ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/master/Master.scala
##########
@@ -245,14 +245,14 @@ 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))

Review comment:
       We don't need to handle the MasterInStandby case anymore ? I guess all messages from worker to master should have this check, right ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
##########
@@ -768,11 +768,13 @@ private[deploy] class Worker(
     }
   }
 
-  private[deploy] def decommissionSelf(): Boolean = {
+  private[deploy] def decommissionSelf(fromMaster: Boolean): Boolean = {
     if (conf.get(config.DECOMMISSION_ENABLED)) {
       logDebug("Decommissioning self")
       decommissioned = true
-      sendToMaster(WorkerDecommission(workerId, self))
+      if (!fromMaster) {
+        sendToMaster(WorkerDecommissioned(workerId))

Review comment:
       I wonder if it is trite to add a comment like: "No need to notify the master if the decommission message already came from it"

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala
##########
@@ -95,8 +95,13 @@ 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 been
+  // used. It's used for the case where decommission is triggered at executor (e.g., K8S)
+  case class ExecutorDecommissioned(executorId: String)
+
+  // A message that sent from driver to executor to decommission that executor.
+  // It's used for Standalone's case yet, where decommission is triggered at Worker.

Review comment:
       remove 'yet'. Are you sure that this is when the decommission is triggered at the Worker ? I think it can also be used when the decommissioning is triggered at the MasterWebUI (not necessarily a SIGPWR at the Worker).

##########
File path: core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
##########
@@ -166,17 +166,6 @@ private[spark] class CoarseGrainedExecutorBackend(
       if (executor == null) {
         exitExecutor(1, "Received LaunchTask command but executor was null")
       } else {
-        if (decommissioned) {

Review comment:
       Are you sure that this case won't happen ? Or perhaps you mean that even if it does happen, it only lasts for a short duration: (until the Driver gets to know that the executor is decommissioned), that its not worth handling it ? If so, I agree with you. 
   
   But just want to confirm when can this case even happen: task launched on a decommissioned executor ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/master/Master.scala
##########
@@ -891,16 +891,13 @@ 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
   }
 
-  private def decommissionWorker(worker: WorkerInfo): Unit = {
+  private def decommissionWorker(worker: WorkerInfo, sentFromWorker: Boolean): Unit = {

Review comment:
       Would triggeredByWorker be a better name than sentFromWorker ?

##########
File path: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
##########
@@ -581,7 +581,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,
+          decommissionFromDriver = true)

Review comment:
       I think decommissionFromDriver = false means that decommission is triggered by a SIGPWR on the executor. 
   
   Whereas, decommissionFromDriver = true means that it might be triggered form either the Master or the Dynamic allocation manager. 
   
   If so, should we flip decommissionFromDriver and just call it decomTriggeredByExecutor ? 

##########
File path: core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
##########
@@ -768,11 +768,13 @@ private[deploy] class Worker(
     }
   }
 
-  private[deploy] def decommissionSelf(): Boolean = {
+  private[deploy] def decommissionSelf(fromMaster: Boolean): Boolean = {

Review comment:
       For consistency: triggeredByMaster instead of fromMaster ?

##########
File path: core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
##########
@@ -264,17 +253,21 @@ private[spark] class CoarseGrainedExecutorBackend(
     System.exit(code)
   }
 
-  private def decommissionSelf(): Boolean = {
+  private def decommissionSelf(fromDriver: Boolean): Boolean = {
     val msg = "Decommissioning self w/sync"
     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)) {

Review comment:
       I see, so is this the now ONLY place where block manager decommissioning is triggered ? Good.

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -272,10 +268,16 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
         removeWorker(workerId, host, message)
         context.reply(true)
 
-      case DecommissionExecutor(executorId, decommissionInfo) =>
-        logError(s"Received decommission executor message ${executorId}: ${decommissionInfo}.")
-        context.reply(decommissionExecutor(executorId, decommissionInfo,
-          adjustTargetNumExecutors = false))
+      case ExecutorDecommissioned(executorId) =>
+        logWarning(s"Received executor $executorId decommissioned message")
+        context.reply(
+          decommissionExecutor(
+            executorId,
+            ExecutorDecommissionInfo(s"Executor $executorId is decommissioned."),
+            adjustTargetNumExecutors = false,
+            // TODO: add a new type like `ExecutorDecommissionInfo` for the case where executor

Review comment:
       This is a good TODO comment, should we add this at the definition of the `decommissionExecutor` method instead of at the caller ?

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {

Review comment:
       nice ! I like the change to flatMap and withLock.

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -1809,7 +1811,7 @@ private[spark] class BlockManager(
     blocksToRemove.size
   }
 
-  def decommissionBlockManager(): Unit = synchronized {
+  private[spark] def decommissionSelf(): Unit = synchronized {

Review comment:
       Didn't follow the need for the name change.

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)
 
-  private def doDecommission(executorId: String,
-      decomInfo: ExecutorDecommissionInfo): Boolean = {
-
-    logInfo(s"Asking executor $executorId to decommissioning.")
-    scheduler.executorDecommission(executorId, decomInfo)
-    // Send decommission message to the executor (it could have originated on the executor
-    // but not necessarily).
-    CoarseGrainedSchedulerBackend.this.synchronized {
-      executorDataMap.get(executorId) match {
-        case Some(executorInfo) =>
-          executorInfo.executorEndpoint.send(DecommissionSelf)
-        case None =>
-          // Ignoring the executor since it is not registered.
-          logWarning(s"Attempted to decommission unknown executor $executorId.")
-          return false
-      }
-    }
-    logInfo(s"Asked executor $executorId to decommission.")
-
-    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
-      try {
-        logInfo(s"Asking block manager corresponding to executor $executorId to decommission.")
-        scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))
-      } catch {
-        case e: Exception =>
-          logError("Unexpected error during block manager " +
-            s"decommissioning for executor $executorId: ${e.toString}", e)
-          return false
+    if (decommissionFromDriver) {
+      CoarseGrainedSchedulerBackend.this.synchronized {

Review comment:
       Why the lock here ? It wasn't locked before (I think).

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)
 
-  private def doDecommission(executorId: String,
-      decomInfo: ExecutorDecommissionInfo): Boolean = {
-
-    logInfo(s"Asking executor $executorId to decommissioning.")
-    scheduler.executorDecommission(executorId, decomInfo)
-    // Send decommission message to the executor (it could have originated on the executor
-    // but not necessarily).
-    CoarseGrainedSchedulerBackend.this.synchronized {
-      executorDataMap.get(executorId) match {
-        case Some(executorInfo) =>
-          executorInfo.executorEndpoint.send(DecommissionSelf)
-        case None =>
-          // Ignoring the executor since it is not registered.
-          logWarning(s"Attempted to decommission unknown executor $executorId.")
-          return false
-      }
-    }
-    logInfo(s"Asked executor $executorId to decommission.")
-
-    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
-      try {
-        logInfo(s"Asking block manager corresponding to executor $executorId to decommission.")
-        scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))
-      } catch {
-        case e: Exception =>
-          logError("Unexpected error during block manager " +
-            s"decommissioning for executor $executorId: ${e.toString}", e)
-          return false
+    if (decommissionFromDriver) {
+      CoarseGrainedSchedulerBackend.this.synchronized {
+        executorsToDecommission.foreach { executorId =>
+          logInfo(s"Asking executor $executorId to decommissioning.")
+          executorDataMap(executorId).executorEndpoint.send(DecommissionExecutor)

Review comment:
        executorDataMap(executorId) might be None: if an executor was removed b/w the end of withLock on L485 to L500. The original code handled that. Should we handle it to ? (We don't have to log anything, just not crash the driver).

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)

Review comment:
       Should this check be guarded by `conf.get(STORAGE_DECOMMISSION_ENABLED)` ? Also should this only be done in case of decommissionFromDriver == true ?
   

##########
File path: core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala
##########
@@ -31,7 +31,7 @@ import org.apache.spark.util.{RpcUtils, SerializableBuffer, ThreadUtils}
 class WorkerDecommissionSuite extends SparkFunSuite with LocalSparkContext {
 
   override def beforeEach(): Unit = {
-    val conf = new SparkConf().setAppName("test").setMaster("local")

Review comment:
       Is master = "local" the default ? And that's why the .setMaster("local") is removed ?

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)
 
-  private def doDecommission(executorId: String,
-      decomInfo: ExecutorDecommissionInfo): Boolean = {
-
-    logInfo(s"Asking executor $executorId to decommissioning.")
-    scheduler.executorDecommission(executorId, decomInfo)
-    // Send decommission message to the executor (it could have originated on the executor
-    // but not necessarily).
-    CoarseGrainedSchedulerBackend.this.synchronized {
-      executorDataMap.get(executorId) match {
-        case Some(executorInfo) =>
-          executorInfo.executorEndpoint.send(DecommissionSelf)
-        case None =>
-          // Ignoring the executor since it is not registered.
-          logWarning(s"Attempted to decommission unknown executor $executorId.")
-          return false
-      }
-    }
-    logInfo(s"Asked executor $executorId to decommission.")
-
-    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
-      try {
-        logInfo(s"Asking block manager corresponding to executor $executorId to decommission.")
-        scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))

Review comment:
       One of the differences with the old design was that the 'Clearing of the block managers from the block manager master and the actual message to the decommission block manager' happened in one place (in scheduler.sc.env.blockManager.master.decommissionBlockManagers). 
   
   Whereas now these two things are split up: The scheduler.sc.env.blockManager.master.decommissionBlockManagers no longer sends the message, just updates its in memory state.
   
   Is there a problem with this difference ? Is some race opened up or closed ?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r487849987



##########
File path: core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
##########
@@ -94,8 +94,9 @@ private[spark] trait ExecutorAllocationClient {
    * @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] = {

Review comment:
       shall we document this new parameter in this method as well?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r489186319



##########
File path: core/src/main/scala/org/apache/spark/deploy/master/Master.scala
##########
@@ -245,15 +245,24 @@ 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)
+        idToWorker.get(id).foreach(w => decommissionWorker(w))
       }
 
+    case DecommissionWorkers(ids) =>
+      ids.foreach ( id =>

Review comment:
       It has been checked by the caller when handling `DecommissionWorkersOnHosts`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r489185564



##########
File path: core/src/main/scala/org/apache/spark/deploy/master/Master.scala
##########
@@ -245,15 +245,24 @@ 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)
+        idToWorker.get(id).foreach(w => decommissionWorker(w))
       }
 
+    case DecommissionWorkers(ids) =>
+      ids.foreach ( id =>

Review comment:
       shall we skip it for `if (state == RecoveryState.STANDBY)` ?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692066848


   **[Test build #128653 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128653/testReport)** for PR 29722 at commit [`541a6e3`](https://github.com/apache/spark/commit/541a6e3d8c80d152e443d06b436fd6febdd2df13).
    * This patch **fails to build**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-691900431


   > I am wondering if you found any interesting issues that you think we don't have great test coverage for ?
   
   I don't..


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-691847911


   **[Test build #128620 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128620/testReport)** for PR 29722 at commit [`7940137`](https://github.com/apache/spark/commit/7940137fbb44ca6d7bd2ba6c190aebc7599fba71).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692513177


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r487664347



##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)
 
-  private def doDecommission(executorId: String,
-      decomInfo: ExecutorDecommissionInfo): Boolean = {
-
-    logInfo(s"Asking executor $executorId to decommissioning.")
-    scheduler.executorDecommission(executorId, decomInfo)
-    // Send decommission message to the executor (it could have originated on the executor
-    // but not necessarily).
-    CoarseGrainedSchedulerBackend.this.synchronized {
-      executorDataMap.get(executorId) match {
-        case Some(executorInfo) =>
-          executorInfo.executorEndpoint.send(DecommissionSelf)
-        case None =>
-          // Ignoring the executor since it is not registered.
-          logWarning(s"Attempted to decommission unknown executor $executorId.")
-          return false
-      }
-    }
-    logInfo(s"Asked executor $executorId to decommission.")
-
-    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
-      try {
-        logInfo(s"Asking block manager corresponding to executor $executorId to decommission.")
-        scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))

Review comment:
       I think it won't cause problems. Actually, I think the only difference is: for a certain BlockManager, previously, it is always marked as `decommissioned` first and then start block migration, but now block migration could start first in case of the decommission is triggered at executor. But it's ok since the decommissioned executor would only migrate blocks to other executors rather than itself.
   
   Note that for the case where decommission notification sent from driver to executor, `decommissioned` status marking still starts first before the block migration, so it doesn't make any differences.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692175412


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/128656/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r487858334



##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +467,42 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
-        // Only bother decommissioning executors which are alive.
-        if (isExecutorActive(executorId)) {
-          executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
-        } else {
-          false
-        }
+      adjustTargetNumExecutors: Boolean,
+      triggeredByExecutor: Boolean): Seq[String] = withLock {
+    val executorsToDecommission = executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
+      // Only bother decommissioning executors which are alive.
+      if (isExecutorActive(executorId)) {
+        scheduler.executorDecommission(executorId, decomInfo)
+        executorsPendingDecommission(executorId) = decomInfo.workerHost
+        Some(executorId)
+      } else {
+        None
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
-
-  private def doDecommission(executorId: String,
-      decomInfo: ExecutorDecommissionInfo): Boolean = {
-
-    logInfo(s"Asking executor $executorId to decommissioning.")
-    scheduler.executorDecommission(executorId, decomInfo)
-    // Send decommission message to the executor (it could have originated on the executor
-    // but not necessarily).
-    CoarseGrainedSchedulerBackend.this.synchronized {
-      executorDataMap.get(executorId) match {
-        case Some(executorInfo) =>
-          executorInfo.executorEndpoint.send(DecommissionSelf)
-        case None =>
-          // Ignoring the executor since it is not registered.
-          logWarning(s"Attempted to decommission unknown executor $executorId.")
-          return false
-      }
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
+      scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)
     }
-    logInfo(s"Asked executor $executorId to decommission.")
 
-    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
-      try {
-        logInfo(s"Asking block manager corresponding to executor $executorId to decommission.")
-        scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))
-      } catch {
-        case e: Exception =>
-          logError("Unexpected error during block manager " +
-            s"decommissioning for executor $executorId: ${e.toString}", e)
-          return false
+    if (triggeredByExecutor) {
+      executorsToDecommission.foreach { executorId =>
+        logInfo(s"Asking executor $executorId to decommissioning.")
+        executorDataMap(executorId).executorEndpoint.send(DecommissionExecutor)

Review comment:
       There is one branch missing from previous code:
   ```
         case None =>
             // Ignoring the executor since it is not registered.
             logWarning(s"Attempted to decommission unknown executor $executorId.")
             return false
   ```
   
   shall we guard against missing executor here as well?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692095758


   **[Test build #128656 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128656/testReport)** for PR 29722 at commit [`d31f020`](https://github.com/apache/spark/commit/d31f0200c18d722c3357b85265d5c45a44801bf6).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r487659285



##########
File path: core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala
##########
@@ -31,7 +31,7 @@ import org.apache.spark.util.{RpcUtils, SerializableBuffer, ThreadUtils}
 class WorkerDecommissionSuite extends SparkFunSuite with LocalSparkContext {
 
   override def beforeEach(): Unit = {
-    val conf = new SparkConf().setAppName("test").setMaster("local")

Review comment:
       No. See line 37 below. `local` is overridden by `local-cluster[2, 1, 1024]`. (I got confused when I was debugging the test failure...so I fixed it.)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692174751


   **[Test build #128656 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128656/testReport)** for PR 29722 at commit [`d31f020`](https://github.com/apache/spark/commit/d31f0200c18d722c3357b85265d5c45a44801bf6).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-693213903


   **[Test build #128748 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128748/testReport)** for PR 29722 at commit [`290d2c0`](https://github.com/apache/spark/commit/290d2c08a1adfce3c0c4afe1cb8c9e214b25ea3d).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692096550






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-693316696






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692064134


   **[Test build #128653 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128653/testReport)** for PR 29722 at commit [`541a6e3`](https://github.com/apache/spark/commit/541a6e3d8c80d152e443d06b436fd6febdd2df13).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692446530






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r487856488



##########
File path: core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
##########
@@ -264,17 +253,21 @@ private[spark] class CoarseGrainedExecutorBackend(
     System.exit(code)
   }
 
-  private def decommissionSelf(): Boolean = {
+  private def decommissionSelf(triggeredByExecutor: Boolean): Boolean = {
     val msg = "Decommissioning self w/sync"
     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()
+      }
+      // Tell driver we starts decommissioning so it stops trying to schedule us
+      if (triggeredByExecutor) {

Review comment:
       similar, we can move this extra code to the caller side, and this method doesn't need the `triggeredByExecutor` parameter.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-695907729


   I didn't realize that the k8s test is in docker and is out of the PR builder. This commit has been reverted. @Ngone51 please resubmit and fix the k8s test.
   
   We should probably improve test coverage for k8s decommission as well.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-693319387


   **[Test build #128763 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128763/testReport)** for PR 29722 at commit [`290d2c0`](https://github.com/apache/spark/commit/290d2c08a1adfce3c0c4afe1cb8c9e214b25ea3d).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simply the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-691026898






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29722: [SPARK-32850][CORE] Simply the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-690851353






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-693442831


   **[Test build #128763 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128763/testReport)** for PR 29722 at commit [`290d2c0`](https://github.com/apache/spark/commit/290d2c08a1adfce3c0c4afe1cb8c9e214b25ea3d).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-691985585


   **[Test build #128645 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128645/testReport)** for PR 29722 at commit [`8f6e279`](https://github.com/apache/spark/commit/8f6e279d5c4c0b2ac6acbe0a4ba88b82827cab61).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-691986108






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692071595


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/128654/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692175405






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692071584






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692096550






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #29722: [SPARK-32850][CORE] Simply the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-690851353






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-693216761


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r489186794



##########
File path: core/src/main/scala/org/apache/spark/deploy/master/Master.scala
##########
@@ -245,15 +245,24 @@ 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)
+        idToWorker.get(id).foreach(w => decommissionWorker(w))
       }
 
+    case DecommissionWorkers(ids) =>
+      ids.foreach ( id =>

Review comment:
       then probably it's better to add an `assert(state != RecoveryState.STANDBY)` here




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r487660636



##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)
 
-  private def doDecommission(executorId: String,
-      decomInfo: ExecutorDecommissionInfo): Boolean = {
-
-    logInfo(s"Asking executor $executorId to decommissioning.")
-    scheduler.executorDecommission(executorId, decomInfo)
-    // Send decommission message to the executor (it could have originated on the executor
-    // but not necessarily).
-    CoarseGrainedSchedulerBackend.this.synchronized {
-      executorDataMap.get(executorId) match {
-        case Some(executorInfo) =>
-          executorInfo.executorEndpoint.send(DecommissionSelf)
-        case None =>
-          // Ignoring the executor since it is not registered.
-          logWarning(s"Attempted to decommission unknown executor $executorId.")
-          return false
-      }
-    }
-    logInfo(s"Asked executor $executorId to decommission.")
-
-    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
-      try {
-        logInfo(s"Asking block manager corresponding to executor $executorId to decommission.")
-        scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))
-      } catch {
-        case e: Exception =>
-          logError("Unexpected error during block manager " +
-            s"decommissioning for executor $executorId: ${e.toString}", e)
-          return false
+    if (decommissionFromDriver) {
+      CoarseGrainedSchedulerBackend.this.synchronized {

Review comment:
       Any access to `executorDataMap` out of the inherited methods from the `CoarseGrainedSchedulerBackend` (endpoint) needs the protection of `CoarseGrainedSchedulerBackend`. See:
   
   https://github.com/apache/spark/blob/742fcff3501e46722eeaeb9d1ac20e569f8f1c2c/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala#L72-L76
   
   
   It also had the lock before? See removed line 502.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r487937373



##########
File path: core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
##########
@@ -79,12 +79,23 @@ 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.send(DecommissionExecutor)

Review comment:
       @cloud-fan @agrawaldevesh What do you think about the latest change? In case of executor is decommissioned by the signal, we send `DecommissionExecutor`(it's a local RPC call) to the executor itself instead of invoking `decommissionSelf` directly.
   
   In this way, we can guarantee the thread-safe of `decommissioned`.
   
   (It's same to the Worker.)
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692071584


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simply the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-690894733






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29722: [SPARK-32850][CORE] Simply the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-690894221


   **[Test build #128548 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128548/testReport)** for PR 29722 at commit [`3bca7c0`](https://github.com/apache/spark/commit/3bca7c001ed0fad7437b566e9c59132209755ee4).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `  case class DecommissionWorkers(ids: Seq[String])`
     * `  case class WorkerDecommissioned(id: String) extends DeployMessage`
     * `  case class ExecutorDecommissioned(executorId: String)`


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29722: [SPARK-32850][CORE] Simply the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-691026898






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692175405


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan closed pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
cloud-fan closed pull request #29722:
URL: https://github.com/apache/spark/pull/29722


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] agrawaldevesh commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
agrawaldevesh commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r487349021



##########
File path: core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
##########
@@ -94,8 +94,9 @@ private[spark] trait ExecutorAllocationClient {
    * @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)],

Review comment:
       I keep messing this up myself, but is this indentation style change intentional ? If 2 spaces are okay then can we remain at that ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
##########
@@ -61,13 +61,28 @@ 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 are pending to be decommissioned.

Review comment:
       nit: which are pending to be decommissioned -> which should be decommissioned ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/master/Master.scala
##########
@@ -245,14 +245,14 @@ 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))

Review comment:
       We don't need to handle the MasterInStandby case anymore ? I guess all messages from worker to master should have this check, right ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
##########
@@ -768,11 +768,13 @@ private[deploy] class Worker(
     }
   }
 
-  private[deploy] def decommissionSelf(): Boolean = {
+  private[deploy] def decommissionSelf(fromMaster: Boolean): Boolean = {
     if (conf.get(config.DECOMMISSION_ENABLED)) {
       logDebug("Decommissioning self")
       decommissioned = true
-      sendToMaster(WorkerDecommission(workerId, self))
+      if (!fromMaster) {
+        sendToMaster(WorkerDecommissioned(workerId))

Review comment:
       I wonder if it is trite to add a comment like: "No need to notify the master if the decommission message already came from it"

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala
##########
@@ -95,8 +95,13 @@ 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 been
+  // used. It's used for the case where decommission is triggered at executor (e.g., K8S)
+  case class ExecutorDecommissioned(executorId: String)
+
+  // A message that sent from driver to executor to decommission that executor.
+  // It's used for Standalone's case yet, where decommission is triggered at Worker.

Review comment:
       remove 'yet'. Are you sure that this is when the decommission is triggered at the Worker ? I think it can also be used when the decommissioning is triggered at the MasterWebUI (not necessarily a SIGPWR at the Worker).

##########
File path: core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
##########
@@ -166,17 +166,6 @@ private[spark] class CoarseGrainedExecutorBackend(
       if (executor == null) {
         exitExecutor(1, "Received LaunchTask command but executor was null")
       } else {
-        if (decommissioned) {

Review comment:
       Are you sure that this case won't happen ? Or perhaps you mean that even if it does happen, it only lasts for a short duration: (until the Driver gets to know that the executor is decommissioned), that its not worth handling it ? If so, I agree with you. 
   
   But just want to confirm when can this case even happen: task launched on a decommissioned executor ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/master/Master.scala
##########
@@ -891,16 +891,13 @@ 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
   }
 
-  private def decommissionWorker(worker: WorkerInfo): Unit = {
+  private def decommissionWorker(worker: WorkerInfo, sentFromWorker: Boolean): Unit = {

Review comment:
       Would triggeredByWorker be a better name than sentFromWorker ?

##########
File path: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
##########
@@ -581,7 +581,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,
+          decommissionFromDriver = true)

Review comment:
       I think decommissionFromDriver = false means that decommission is triggered by a SIGPWR on the executor. 
   
   Whereas, decommissionFromDriver = true means that it might be triggered form either the Master or the Dynamic allocation manager. 
   
   If so, should we flip decommissionFromDriver and just call it decomTriggeredByExecutor ? 

##########
File path: core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
##########
@@ -768,11 +768,13 @@ private[deploy] class Worker(
     }
   }
 
-  private[deploy] def decommissionSelf(): Boolean = {
+  private[deploy] def decommissionSelf(fromMaster: Boolean): Boolean = {

Review comment:
       For consistency: triggeredByMaster instead of fromMaster ?

##########
File path: core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
##########
@@ -264,17 +253,21 @@ private[spark] class CoarseGrainedExecutorBackend(
     System.exit(code)
   }
 
-  private def decommissionSelf(): Boolean = {
+  private def decommissionSelf(fromDriver: Boolean): Boolean = {
     val msg = "Decommissioning self w/sync"
     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)) {

Review comment:
       I see, so is this the now ONLY place where block manager decommissioning is triggered ? Good.

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -272,10 +268,16 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
         removeWorker(workerId, host, message)
         context.reply(true)
 
-      case DecommissionExecutor(executorId, decommissionInfo) =>
-        logError(s"Received decommission executor message ${executorId}: ${decommissionInfo}.")
-        context.reply(decommissionExecutor(executorId, decommissionInfo,
-          adjustTargetNumExecutors = false))
+      case ExecutorDecommissioned(executorId) =>
+        logWarning(s"Received executor $executorId decommissioned message")
+        context.reply(
+          decommissionExecutor(
+            executorId,
+            ExecutorDecommissionInfo(s"Executor $executorId is decommissioned."),
+            adjustTargetNumExecutors = false,
+            // TODO: add a new type like `ExecutorDecommissionInfo` for the case where executor

Review comment:
       This is a good TODO comment, should we add this at the definition of the `decommissionExecutor` method instead of at the caller ?

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {

Review comment:
       nice ! I like the change to flatMap and withLock.

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -1809,7 +1811,7 @@ private[spark] class BlockManager(
     blocksToRemove.size
   }
 
-  def decommissionBlockManager(): Unit = synchronized {
+  private[spark] def decommissionSelf(): Unit = synchronized {

Review comment:
       Didn't follow the need for the name change.

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)
 
-  private def doDecommission(executorId: String,
-      decomInfo: ExecutorDecommissionInfo): Boolean = {
-
-    logInfo(s"Asking executor $executorId to decommissioning.")
-    scheduler.executorDecommission(executorId, decomInfo)
-    // Send decommission message to the executor (it could have originated on the executor
-    // but not necessarily).
-    CoarseGrainedSchedulerBackend.this.synchronized {
-      executorDataMap.get(executorId) match {
-        case Some(executorInfo) =>
-          executorInfo.executorEndpoint.send(DecommissionSelf)
-        case None =>
-          // Ignoring the executor since it is not registered.
-          logWarning(s"Attempted to decommission unknown executor $executorId.")
-          return false
-      }
-    }
-    logInfo(s"Asked executor $executorId to decommission.")
-
-    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
-      try {
-        logInfo(s"Asking block manager corresponding to executor $executorId to decommission.")
-        scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))
-      } catch {
-        case e: Exception =>
-          logError("Unexpected error during block manager " +
-            s"decommissioning for executor $executorId: ${e.toString}", e)
-          return false
+    if (decommissionFromDriver) {
+      CoarseGrainedSchedulerBackend.this.synchronized {

Review comment:
       Why the lock here ? It wasn't locked before (I think).

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)
 
-  private def doDecommission(executorId: String,
-      decomInfo: ExecutorDecommissionInfo): Boolean = {
-
-    logInfo(s"Asking executor $executorId to decommissioning.")
-    scheduler.executorDecommission(executorId, decomInfo)
-    // Send decommission message to the executor (it could have originated on the executor
-    // but not necessarily).
-    CoarseGrainedSchedulerBackend.this.synchronized {
-      executorDataMap.get(executorId) match {
-        case Some(executorInfo) =>
-          executorInfo.executorEndpoint.send(DecommissionSelf)
-        case None =>
-          // Ignoring the executor since it is not registered.
-          logWarning(s"Attempted to decommission unknown executor $executorId.")
-          return false
-      }
-    }
-    logInfo(s"Asked executor $executorId to decommission.")
-
-    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
-      try {
-        logInfo(s"Asking block manager corresponding to executor $executorId to decommission.")
-        scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))
-      } catch {
-        case e: Exception =>
-          logError("Unexpected error during block manager " +
-            s"decommissioning for executor $executorId: ${e.toString}", e)
-          return false
+    if (decommissionFromDriver) {
+      CoarseGrainedSchedulerBackend.this.synchronized {
+        executorsToDecommission.foreach { executorId =>
+          logInfo(s"Asking executor $executorId to decommissioning.")
+          executorDataMap(executorId).executorEndpoint.send(DecommissionExecutor)

Review comment:
        executorDataMap(executorId) might be None: if an executor was removed b/w the end of withLock on L485 to L500. The original code handled that. Should we handle it to ? (We don't have to log anything, just not crash the driver).

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)

Review comment:
       Should this check be guarded by `conf.get(STORAGE_DECOMMISSION_ENABLED)` ? Also should this only be done in case of decommissionFromDriver == true ?
   

##########
File path: core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala
##########
@@ -31,7 +31,7 @@ import org.apache.spark.util.{RpcUtils, SerializableBuffer, ThreadUtils}
 class WorkerDecommissionSuite extends SparkFunSuite with LocalSparkContext {
 
   override def beforeEach(): Unit = {
-    val conf = new SparkConf().setAppName("test").setMaster("local")

Review comment:
       Is master = "local" the default ? And that's why the .setMaster("local") is removed ?

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)
 
-  private def doDecommission(executorId: String,
-      decomInfo: ExecutorDecommissionInfo): Boolean = {
-
-    logInfo(s"Asking executor $executorId to decommissioning.")
-    scheduler.executorDecommission(executorId, decomInfo)
-    // Send decommission message to the executor (it could have originated on the executor
-    // but not necessarily).
-    CoarseGrainedSchedulerBackend.this.synchronized {
-      executorDataMap.get(executorId) match {
-        case Some(executorInfo) =>
-          executorInfo.executorEndpoint.send(DecommissionSelf)
-        case None =>
-          // Ignoring the executor since it is not registered.
-          logWarning(s"Attempted to decommission unknown executor $executorId.")
-          return false
-      }
-    }
-    logInfo(s"Asked executor $executorId to decommission.")
-
-    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
-      try {
-        logInfo(s"Asking block manager corresponding to executor $executorId to decommission.")
-        scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))

Review comment:
       One of the differences with the old design was that the 'Clearing of the block managers from the block manager master and the actual message to the decommission block manager' happened in one place (in scheduler.sc.env.blockManager.master.decommissionBlockManagers). 
   
   Whereas now these two things are split up: The scheduler.sc.env.blockManager.master.decommissionBlockManagers no longer sends the message, just updates its in memory state.
   
   Is there a problem with this difference ? Is some race opened up or closed ?

##########
File path: core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
##########
@@ -94,8 +94,9 @@ private[spark] trait ExecutorAllocationClient {
    * @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)],

Review comment:
       I keep messing this up myself, but is this indentation style change intentional ? If 2 spaces are okay then can we remain at that ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
##########
@@ -61,13 +61,28 @@ 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 are pending to be decommissioned.

Review comment:
       nit: which are pending to be decommissioned -> which should be decommissioned ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/master/Master.scala
##########
@@ -245,14 +245,14 @@ 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))

Review comment:
       We don't need to handle the MasterInStandby case anymore ? I guess all messages from worker to master should have this check, right ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
##########
@@ -768,11 +768,13 @@ private[deploy] class Worker(
     }
   }
 
-  private[deploy] def decommissionSelf(): Boolean = {
+  private[deploy] def decommissionSelf(fromMaster: Boolean): Boolean = {
     if (conf.get(config.DECOMMISSION_ENABLED)) {
       logDebug("Decommissioning self")
       decommissioned = true
-      sendToMaster(WorkerDecommission(workerId, self))
+      if (!fromMaster) {
+        sendToMaster(WorkerDecommissioned(workerId))

Review comment:
       I wonder if it is trite to add a comment like: "No need to notify the master if the decommission message already came from it"

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala
##########
@@ -95,8 +95,13 @@ 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 been
+  // used. It's used for the case where decommission is triggered at executor (e.g., K8S)
+  case class ExecutorDecommissioned(executorId: String)
+
+  // A message that sent from driver to executor to decommission that executor.
+  // It's used for Standalone's case yet, where decommission is triggered at Worker.

Review comment:
       remove 'yet'. Are you sure that this is when the decommission is triggered at the Worker ? I think it can also be used when the decommissioning is triggered at the MasterWebUI (not necessarily a SIGPWR at the Worker).

##########
File path: core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
##########
@@ -166,17 +166,6 @@ private[spark] class CoarseGrainedExecutorBackend(
       if (executor == null) {
         exitExecutor(1, "Received LaunchTask command but executor was null")
       } else {
-        if (decommissioned) {

Review comment:
       Are you sure that this case won't happen ? Or perhaps you mean that even if it does happen, it only lasts for a short duration: (until the Driver gets to know that the executor is decommissioned), that its not worth handling it ? If so, I agree with you. 
   
   But just want to confirm when can this case even happen: task launched on a decommissioned executor ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/master/Master.scala
##########
@@ -891,16 +891,13 @@ 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
   }
 
-  private def decommissionWorker(worker: WorkerInfo): Unit = {
+  private def decommissionWorker(worker: WorkerInfo, sentFromWorker: Boolean): Unit = {

Review comment:
       Would triggeredByWorker be a better name than sentFromWorker ?

##########
File path: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
##########
@@ -581,7 +581,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,
+          decommissionFromDriver = true)

Review comment:
       I think decommissionFromDriver = false means that decommission is triggered by a SIGPWR on the executor. 
   
   Whereas, decommissionFromDriver = true means that it might be triggered form either the Master or the Dynamic allocation manager. 
   
   If so, should we flip decommissionFromDriver and just call it decomTriggeredByExecutor ? 

##########
File path: core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
##########
@@ -768,11 +768,13 @@ private[deploy] class Worker(
     }
   }
 
-  private[deploy] def decommissionSelf(): Boolean = {
+  private[deploy] def decommissionSelf(fromMaster: Boolean): Boolean = {

Review comment:
       For consistency: triggeredByMaster instead of fromMaster ?

##########
File path: core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
##########
@@ -264,17 +253,21 @@ private[spark] class CoarseGrainedExecutorBackend(
     System.exit(code)
   }
 
-  private def decommissionSelf(): Boolean = {
+  private def decommissionSelf(fromDriver: Boolean): Boolean = {
     val msg = "Decommissioning self w/sync"
     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)) {

Review comment:
       I see, so is this the now ONLY place where block manager decommissioning is triggered ? Good.

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -272,10 +268,16 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
         removeWorker(workerId, host, message)
         context.reply(true)
 
-      case DecommissionExecutor(executorId, decommissionInfo) =>
-        logError(s"Received decommission executor message ${executorId}: ${decommissionInfo}.")
-        context.reply(decommissionExecutor(executorId, decommissionInfo,
-          adjustTargetNumExecutors = false))
+      case ExecutorDecommissioned(executorId) =>
+        logWarning(s"Received executor $executorId decommissioned message")
+        context.reply(
+          decommissionExecutor(
+            executorId,
+            ExecutorDecommissionInfo(s"Executor $executorId is decommissioned."),
+            adjustTargetNumExecutors = false,
+            // TODO: add a new type like `ExecutorDecommissionInfo` for the case where executor

Review comment:
       This is a good TODO comment, should we add this at the definition of the `decommissionExecutor` method instead of at the caller ?

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {

Review comment:
       nice ! I like the change to flatMap and withLock.

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -1809,7 +1811,7 @@ private[spark] class BlockManager(
     blocksToRemove.size
   }
 
-  def decommissionBlockManager(): Unit = synchronized {
+  private[spark] def decommissionSelf(): Unit = synchronized {

Review comment:
       Didn't follow the need for the name change.

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)
 
-  private def doDecommission(executorId: String,
-      decomInfo: ExecutorDecommissionInfo): Boolean = {
-
-    logInfo(s"Asking executor $executorId to decommissioning.")
-    scheduler.executorDecommission(executorId, decomInfo)
-    // Send decommission message to the executor (it could have originated on the executor
-    // but not necessarily).
-    CoarseGrainedSchedulerBackend.this.synchronized {
-      executorDataMap.get(executorId) match {
-        case Some(executorInfo) =>
-          executorInfo.executorEndpoint.send(DecommissionSelf)
-        case None =>
-          // Ignoring the executor since it is not registered.
-          logWarning(s"Attempted to decommission unknown executor $executorId.")
-          return false
-      }
-    }
-    logInfo(s"Asked executor $executorId to decommission.")
-
-    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
-      try {
-        logInfo(s"Asking block manager corresponding to executor $executorId to decommission.")
-        scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))
-      } catch {
-        case e: Exception =>
-          logError("Unexpected error during block manager " +
-            s"decommissioning for executor $executorId: ${e.toString}", e)
-          return false
+    if (decommissionFromDriver) {
+      CoarseGrainedSchedulerBackend.this.synchronized {

Review comment:
       Why the lock here ? It wasn't locked before (I think).

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)
 
-  private def doDecommission(executorId: String,
-      decomInfo: ExecutorDecommissionInfo): Boolean = {
-
-    logInfo(s"Asking executor $executorId to decommissioning.")
-    scheduler.executorDecommission(executorId, decomInfo)
-    // Send decommission message to the executor (it could have originated on the executor
-    // but not necessarily).
-    CoarseGrainedSchedulerBackend.this.synchronized {
-      executorDataMap.get(executorId) match {
-        case Some(executorInfo) =>
-          executorInfo.executorEndpoint.send(DecommissionSelf)
-        case None =>
-          // Ignoring the executor since it is not registered.
-          logWarning(s"Attempted to decommission unknown executor $executorId.")
-          return false
-      }
-    }
-    logInfo(s"Asked executor $executorId to decommission.")
-
-    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
-      try {
-        logInfo(s"Asking block manager corresponding to executor $executorId to decommission.")
-        scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))
-      } catch {
-        case e: Exception =>
-          logError("Unexpected error during block manager " +
-            s"decommissioning for executor $executorId: ${e.toString}", e)
-          return false
+    if (decommissionFromDriver) {
+      CoarseGrainedSchedulerBackend.this.synchronized {
+        executorsToDecommission.foreach { executorId =>
+          logInfo(s"Asking executor $executorId to decommissioning.")
+          executorDataMap(executorId).executorEndpoint.send(DecommissionExecutor)

Review comment:
        executorDataMap(executorId) might be None: if an executor was removed b/w the end of withLock on L485 to L500. The original code handled that. Should we handle it to ? (We don't have to log anything, just not crash the driver).

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)

Review comment:
       Should this check be guarded by `conf.get(STORAGE_DECOMMISSION_ENABLED)` ? Also should this only be done in case of decommissionFromDriver == true ?
   

##########
File path: core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala
##########
@@ -31,7 +31,7 @@ import org.apache.spark.util.{RpcUtils, SerializableBuffer, ThreadUtils}
 class WorkerDecommissionSuite extends SparkFunSuite with LocalSparkContext {
 
   override def beforeEach(): Unit = {
-    val conf = new SparkConf().setAppName("test").setMaster("local")

Review comment:
       Is master = "local" the default ? And that's why the .setMaster("local") is removed ?

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)
 
-  private def doDecommission(executorId: String,
-      decomInfo: ExecutorDecommissionInfo): Boolean = {
-
-    logInfo(s"Asking executor $executorId to decommissioning.")
-    scheduler.executorDecommission(executorId, decomInfo)
-    // Send decommission message to the executor (it could have originated on the executor
-    // but not necessarily).
-    CoarseGrainedSchedulerBackend.this.synchronized {
-      executorDataMap.get(executorId) match {
-        case Some(executorInfo) =>
-          executorInfo.executorEndpoint.send(DecommissionSelf)
-        case None =>
-          // Ignoring the executor since it is not registered.
-          logWarning(s"Attempted to decommission unknown executor $executorId.")
-          return false
-      }
-    }
-    logInfo(s"Asked executor $executorId to decommission.")
-
-    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
-      try {
-        logInfo(s"Asking block manager corresponding to executor $executorId to decommission.")
-        scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))

Review comment:
       One of the differences with the old design was that the 'Clearing of the block managers from the block manager master and the actual message to the decommission block manager' happened in one place (in scheduler.sc.env.blockManager.master.decommissionBlockManagers). 
   
   Whereas now these two things are split up: The scheduler.sc.env.blockManager.master.decommissionBlockManagers no longer sends the message, just updates its in memory state.
   
   Is there a problem with this difference ? Is some race opened up or closed ?

##########
File path: core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
##########
@@ -94,8 +94,9 @@ private[spark] trait ExecutorAllocationClient {
    * @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)],

Review comment:
       I keep messing this up myself, but is this indentation style change intentional ? If 2 spaces are okay then can we remain at that ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
##########
@@ -61,13 +61,28 @@ 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 are pending to be decommissioned.

Review comment:
       nit: which are pending to be decommissioned -> which should be decommissioned ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/master/Master.scala
##########
@@ -245,14 +245,14 @@ 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))

Review comment:
       We don't need to handle the MasterInStandby case anymore ? I guess all messages from worker to master should have this check, right ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
##########
@@ -768,11 +768,13 @@ private[deploy] class Worker(
     }
   }
 
-  private[deploy] def decommissionSelf(): Boolean = {
+  private[deploy] def decommissionSelf(fromMaster: Boolean): Boolean = {
     if (conf.get(config.DECOMMISSION_ENABLED)) {
       logDebug("Decommissioning self")
       decommissioned = true
-      sendToMaster(WorkerDecommission(workerId, self))
+      if (!fromMaster) {
+        sendToMaster(WorkerDecommissioned(workerId))

Review comment:
       I wonder if it is trite to add a comment like: "No need to notify the master if the decommission message already came from it"

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala
##########
@@ -95,8 +95,13 @@ 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 been
+  // used. It's used for the case where decommission is triggered at executor (e.g., K8S)
+  case class ExecutorDecommissioned(executorId: String)
+
+  // A message that sent from driver to executor to decommission that executor.
+  // It's used for Standalone's case yet, where decommission is triggered at Worker.

Review comment:
       remove 'yet'. Are you sure that this is when the decommission is triggered at the Worker ? I think it can also be used when the decommissioning is triggered at the MasterWebUI (not necessarily a SIGPWR at the Worker).

##########
File path: core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
##########
@@ -166,17 +166,6 @@ private[spark] class CoarseGrainedExecutorBackend(
       if (executor == null) {
         exitExecutor(1, "Received LaunchTask command but executor was null")
       } else {
-        if (decommissioned) {

Review comment:
       Are you sure that this case won't happen ? Or perhaps you mean that even if it does happen, it only lasts for a short duration: (until the Driver gets to know that the executor is decommissioned), that its not worth handling it ? If so, I agree with you. 
   
   But just want to confirm when can this case even happen: task launched on a decommissioned executor ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/master/Master.scala
##########
@@ -891,16 +891,13 @@ 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
   }
 
-  private def decommissionWorker(worker: WorkerInfo): Unit = {
+  private def decommissionWorker(worker: WorkerInfo, sentFromWorker: Boolean): Unit = {

Review comment:
       Would triggeredByWorker be a better name than sentFromWorker ?

##########
File path: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
##########
@@ -581,7 +581,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,
+          decommissionFromDriver = true)

Review comment:
       I think decommissionFromDriver = false means that decommission is triggered by a SIGPWR on the executor. 
   
   Whereas, decommissionFromDriver = true means that it might be triggered form either the Master or the Dynamic allocation manager. 
   
   If so, should we flip decommissionFromDriver and just call it decomTriggeredByExecutor ? 

##########
File path: core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
##########
@@ -768,11 +768,13 @@ private[deploy] class Worker(
     }
   }
 
-  private[deploy] def decommissionSelf(): Boolean = {
+  private[deploy] def decommissionSelf(fromMaster: Boolean): Boolean = {

Review comment:
       For consistency: triggeredByMaster instead of fromMaster ?

##########
File path: core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
##########
@@ -264,17 +253,21 @@ private[spark] class CoarseGrainedExecutorBackend(
     System.exit(code)
   }
 
-  private def decommissionSelf(): Boolean = {
+  private def decommissionSelf(fromDriver: Boolean): Boolean = {
     val msg = "Decommissioning self w/sync"
     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)) {

Review comment:
       I see, so is this the now ONLY place where block manager decommissioning is triggered ? Good.

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -272,10 +268,16 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
         removeWorker(workerId, host, message)
         context.reply(true)
 
-      case DecommissionExecutor(executorId, decommissionInfo) =>
-        logError(s"Received decommission executor message ${executorId}: ${decommissionInfo}.")
-        context.reply(decommissionExecutor(executorId, decommissionInfo,
-          adjustTargetNumExecutors = false))
+      case ExecutorDecommissioned(executorId) =>
+        logWarning(s"Received executor $executorId decommissioned message")
+        context.reply(
+          decommissionExecutor(
+            executorId,
+            ExecutorDecommissionInfo(s"Executor $executorId is decommissioned."),
+            adjustTargetNumExecutors = false,
+            // TODO: add a new type like `ExecutorDecommissionInfo` for the case where executor

Review comment:
       This is a good TODO comment, should we add this at the definition of the `decommissionExecutor` method instead of at the caller ?

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {

Review comment:
       nice ! I like the change to flatMap and withLock.

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -1809,7 +1811,7 @@ private[spark] class BlockManager(
     blocksToRemove.size
   }
 
-  def decommissionBlockManager(): Unit = synchronized {
+  private[spark] def decommissionSelf(): Unit = synchronized {

Review comment:
       Didn't follow the need for the name change.

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)
 
-  private def doDecommission(executorId: String,
-      decomInfo: ExecutorDecommissionInfo): Boolean = {
-
-    logInfo(s"Asking executor $executorId to decommissioning.")
-    scheduler.executorDecommission(executorId, decomInfo)
-    // Send decommission message to the executor (it could have originated on the executor
-    // but not necessarily).
-    CoarseGrainedSchedulerBackend.this.synchronized {
-      executorDataMap.get(executorId) match {
-        case Some(executorInfo) =>
-          executorInfo.executorEndpoint.send(DecommissionSelf)
-        case None =>
-          // Ignoring the executor since it is not registered.
-          logWarning(s"Attempted to decommission unknown executor $executorId.")
-          return false
-      }
-    }
-    logInfo(s"Asked executor $executorId to decommission.")
-
-    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
-      try {
-        logInfo(s"Asking block manager corresponding to executor $executorId to decommission.")
-        scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))
-      } catch {
-        case e: Exception =>
-          logError("Unexpected error during block manager " +
-            s"decommissioning for executor $executorId: ${e.toString}", e)
-          return false
+    if (decommissionFromDriver) {
+      CoarseGrainedSchedulerBackend.this.synchronized {

Review comment:
       Why the lock here ? It wasn't locked before (I think).

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)
 
-  private def doDecommission(executorId: String,
-      decomInfo: ExecutorDecommissionInfo): Boolean = {
-
-    logInfo(s"Asking executor $executorId to decommissioning.")
-    scheduler.executorDecommission(executorId, decomInfo)
-    // Send decommission message to the executor (it could have originated on the executor
-    // but not necessarily).
-    CoarseGrainedSchedulerBackend.this.synchronized {
-      executorDataMap.get(executorId) match {
-        case Some(executorInfo) =>
-          executorInfo.executorEndpoint.send(DecommissionSelf)
-        case None =>
-          // Ignoring the executor since it is not registered.
-          logWarning(s"Attempted to decommission unknown executor $executorId.")
-          return false
-      }
-    }
-    logInfo(s"Asked executor $executorId to decommission.")
-
-    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
-      try {
-        logInfo(s"Asking block manager corresponding to executor $executorId to decommission.")
-        scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))
-      } catch {
-        case e: Exception =>
-          logError("Unexpected error during block manager " +
-            s"decommissioning for executor $executorId: ${e.toString}", e)
-          return false
+    if (decommissionFromDriver) {
+      CoarseGrainedSchedulerBackend.this.synchronized {
+        executorsToDecommission.foreach { executorId =>
+          logInfo(s"Asking executor $executorId to decommissioning.")
+          executorDataMap(executorId).executorEndpoint.send(DecommissionExecutor)

Review comment:
        executorDataMap(executorId) might be None: if an executor was removed b/w the end of withLock on L485 to L500. The original code handled that. Should we handle it to ? (We don't have to log anything, just not crash the driver).

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)

Review comment:
       Should this check be guarded by `conf.get(STORAGE_DECOMMISSION_ENABLED)` ? Also should this only be done in case of decommissionFromDriver == true ?
   

##########
File path: core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala
##########
@@ -31,7 +31,7 @@ import org.apache.spark.util.{RpcUtils, SerializableBuffer, ThreadUtils}
 class WorkerDecommissionSuite extends SparkFunSuite with LocalSparkContext {
 
   override def beforeEach(): Unit = {
-    val conf = new SparkConf().setAppName("test").setMaster("local")

Review comment:
       Is master = "local" the default ? And that's why the .setMaster("local") is removed ?

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)
 
-  private def doDecommission(executorId: String,
-      decomInfo: ExecutorDecommissionInfo): Boolean = {
-
-    logInfo(s"Asking executor $executorId to decommissioning.")
-    scheduler.executorDecommission(executorId, decomInfo)
-    // Send decommission message to the executor (it could have originated on the executor
-    // but not necessarily).
-    CoarseGrainedSchedulerBackend.this.synchronized {
-      executorDataMap.get(executorId) match {
-        case Some(executorInfo) =>
-          executorInfo.executorEndpoint.send(DecommissionSelf)
-        case None =>
-          // Ignoring the executor since it is not registered.
-          logWarning(s"Attempted to decommission unknown executor $executorId.")
-          return false
-      }
-    }
-    logInfo(s"Asked executor $executorId to decommission.")
-
-    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
-      try {
-        logInfo(s"Asking block manager corresponding to executor $executorId to decommission.")
-        scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))

Review comment:
       One of the differences with the old design was that the 'Clearing of the block managers from the block manager master and the actual message to the decommission block manager' happened in one place (in scheduler.sc.env.blockManager.master.decommissionBlockManagers). 
   
   Whereas now these two things are split up: The scheduler.sc.env.blockManager.master.decommissionBlockManagers no longer sends the message, just updates its in memory state.
   
   Is there a problem with this difference ? Is some race opened up or closed ?

##########
File path: core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
##########
@@ -94,8 +94,9 @@ private[spark] trait ExecutorAllocationClient {
    * @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)],

Review comment:
       I keep messing this up myself, but is this indentation style change intentional ? If 2 spaces are okay then can we remain at that ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
##########
@@ -61,13 +61,28 @@ 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 are pending to be decommissioned.

Review comment:
       nit: which are pending to be decommissioned -> which should be decommissioned ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/master/Master.scala
##########
@@ -245,14 +245,14 @@ 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))

Review comment:
       We don't need to handle the MasterInStandby case anymore ? I guess all messages from worker to master should have this check, right ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
##########
@@ -768,11 +768,13 @@ private[deploy] class Worker(
     }
   }
 
-  private[deploy] def decommissionSelf(): Boolean = {
+  private[deploy] def decommissionSelf(fromMaster: Boolean): Boolean = {
     if (conf.get(config.DECOMMISSION_ENABLED)) {
       logDebug("Decommissioning self")
       decommissioned = true
-      sendToMaster(WorkerDecommission(workerId, self))
+      if (!fromMaster) {
+        sendToMaster(WorkerDecommissioned(workerId))

Review comment:
       I wonder if it is trite to add a comment like: "No need to notify the master if the decommission message already came from it"

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala
##########
@@ -95,8 +95,13 @@ 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 been
+  // used. It's used for the case where decommission is triggered at executor (e.g., K8S)
+  case class ExecutorDecommissioned(executorId: String)
+
+  // A message that sent from driver to executor to decommission that executor.
+  // It's used for Standalone's case yet, where decommission is triggered at Worker.

Review comment:
       remove 'yet'. Are you sure that this is when the decommission is triggered at the Worker ? I think it can also be used when the decommissioning is triggered at the MasterWebUI (not necessarily a SIGPWR at the Worker).

##########
File path: core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
##########
@@ -166,17 +166,6 @@ private[spark] class CoarseGrainedExecutorBackend(
       if (executor == null) {
         exitExecutor(1, "Received LaunchTask command but executor was null")
       } else {
-        if (decommissioned) {

Review comment:
       Are you sure that this case won't happen ? Or perhaps you mean that even if it does happen, it only lasts for a short duration: (until the Driver gets to know that the executor is decommissioned), that its not worth handling it ? If so, I agree with you. 
   
   But just want to confirm when can this case even happen: task launched on a decommissioned executor ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/master/Master.scala
##########
@@ -891,16 +891,13 @@ 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
   }
 
-  private def decommissionWorker(worker: WorkerInfo): Unit = {
+  private def decommissionWorker(worker: WorkerInfo, sentFromWorker: Boolean): Unit = {

Review comment:
       Would triggeredByWorker be a better name than sentFromWorker ?

##########
File path: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
##########
@@ -581,7 +581,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,
+          decommissionFromDriver = true)

Review comment:
       I think decommissionFromDriver = false means that decommission is triggered by a SIGPWR on the executor. 
   
   Whereas, decommissionFromDriver = true means that it might be triggered form either the Master or the Dynamic allocation manager. 
   
   If so, should we flip decommissionFromDriver and just call it decomTriggeredByExecutor ? 

##########
File path: core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
##########
@@ -768,11 +768,13 @@ private[deploy] class Worker(
     }
   }
 
-  private[deploy] def decommissionSelf(): Boolean = {
+  private[deploy] def decommissionSelf(fromMaster: Boolean): Boolean = {

Review comment:
       For consistency: triggeredByMaster instead of fromMaster ?

##########
File path: core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
##########
@@ -264,17 +253,21 @@ private[spark] class CoarseGrainedExecutorBackend(
     System.exit(code)
   }
 
-  private def decommissionSelf(): Boolean = {
+  private def decommissionSelf(fromDriver: Boolean): Boolean = {
     val msg = "Decommissioning self w/sync"
     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)) {

Review comment:
       I see, so is this the now ONLY place where block manager decommissioning is triggered ? Good.

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -272,10 +268,16 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
         removeWorker(workerId, host, message)
         context.reply(true)
 
-      case DecommissionExecutor(executorId, decommissionInfo) =>
-        logError(s"Received decommission executor message ${executorId}: ${decommissionInfo}.")
-        context.reply(decommissionExecutor(executorId, decommissionInfo,
-          adjustTargetNumExecutors = false))
+      case ExecutorDecommissioned(executorId) =>
+        logWarning(s"Received executor $executorId decommissioned message")
+        context.reply(
+          decommissionExecutor(
+            executorId,
+            ExecutorDecommissionInfo(s"Executor $executorId is decommissioned."),
+            adjustTargetNumExecutors = false,
+            // TODO: add a new type like `ExecutorDecommissionInfo` for the case where executor

Review comment:
       This is a good TODO comment, should we add this at the definition of the `decommissionExecutor` method instead of at the caller ?

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {

Review comment:
       nice ! I like the change to flatMap and withLock.

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -1809,7 +1811,7 @@ private[spark] class BlockManager(
     blocksToRemove.size
   }
 
-  def decommissionBlockManager(): Unit = synchronized {
+  private[spark] def decommissionSelf(): Unit = synchronized {

Review comment:
       Didn't follow the need for the name change.

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)
 
-  private def doDecommission(executorId: String,
-      decomInfo: ExecutorDecommissionInfo): Boolean = {
-
-    logInfo(s"Asking executor $executorId to decommissioning.")
-    scheduler.executorDecommission(executorId, decomInfo)
-    // Send decommission message to the executor (it could have originated on the executor
-    // but not necessarily).
-    CoarseGrainedSchedulerBackend.this.synchronized {
-      executorDataMap.get(executorId) match {
-        case Some(executorInfo) =>
-          executorInfo.executorEndpoint.send(DecommissionSelf)
-        case None =>
-          // Ignoring the executor since it is not registered.
-          logWarning(s"Attempted to decommission unknown executor $executorId.")
-          return false
-      }
-    }
-    logInfo(s"Asked executor $executorId to decommission.")
-
-    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
-      try {
-        logInfo(s"Asking block manager corresponding to executor $executorId to decommission.")
-        scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))
-      } catch {
-        case e: Exception =>
-          logError("Unexpected error during block manager " +
-            s"decommissioning for executor $executorId: ${e.toString}", e)
-          return false
+    if (decommissionFromDriver) {
+      CoarseGrainedSchedulerBackend.this.synchronized {

Review comment:
       Why the lock here ? It wasn't locked before (I think).

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)
 
-  private def doDecommission(executorId: String,
-      decomInfo: ExecutorDecommissionInfo): Boolean = {
-
-    logInfo(s"Asking executor $executorId to decommissioning.")
-    scheduler.executorDecommission(executorId, decomInfo)
-    // Send decommission message to the executor (it could have originated on the executor
-    // but not necessarily).
-    CoarseGrainedSchedulerBackend.this.synchronized {
-      executorDataMap.get(executorId) match {
-        case Some(executorInfo) =>
-          executorInfo.executorEndpoint.send(DecommissionSelf)
-        case None =>
-          // Ignoring the executor since it is not registered.
-          logWarning(s"Attempted to decommission unknown executor $executorId.")
-          return false
-      }
-    }
-    logInfo(s"Asked executor $executorId to decommission.")
-
-    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
-      try {
-        logInfo(s"Asking block manager corresponding to executor $executorId to decommission.")
-        scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))
-      } catch {
-        case e: Exception =>
-          logError("Unexpected error during block manager " +
-            s"decommissioning for executor $executorId: ${e.toString}", e)
-          return false
+    if (decommissionFromDriver) {
+      CoarseGrainedSchedulerBackend.this.synchronized {
+        executorsToDecommission.foreach { executorId =>
+          logInfo(s"Asking executor $executorId to decommissioning.")
+          executorDataMap(executorId).executorEndpoint.send(DecommissionExecutor)

Review comment:
        executorDataMap(executorId) might be None: if an executor was removed b/w the end of withLock on L485 to L500. The original code handled that. Should we handle it to ? (We don't have to log anything, just not crash the driver).

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)

Review comment:
       Should this check be guarded by `conf.get(STORAGE_DECOMMISSION_ENABLED)` ? Also should this only be done in case of decommissionFromDriver == true ?
   

##########
File path: core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala
##########
@@ -31,7 +31,7 @@ import org.apache.spark.util.{RpcUtils, SerializableBuffer, ThreadUtils}
 class WorkerDecommissionSuite extends SparkFunSuite with LocalSparkContext {
 
   override def beforeEach(): Unit = {
-    val conf = new SparkConf().setAppName("test").setMaster("local")

Review comment:
       Is master = "local" the default ? And that's why the .setMaster("local") is removed ?

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)
 
-  private def doDecommission(executorId: String,
-      decomInfo: ExecutorDecommissionInfo): Boolean = {
-
-    logInfo(s"Asking executor $executorId to decommissioning.")
-    scheduler.executorDecommission(executorId, decomInfo)
-    // Send decommission message to the executor (it could have originated on the executor
-    // but not necessarily).
-    CoarseGrainedSchedulerBackend.this.synchronized {
-      executorDataMap.get(executorId) match {
-        case Some(executorInfo) =>
-          executorInfo.executorEndpoint.send(DecommissionSelf)
-        case None =>
-          // Ignoring the executor since it is not registered.
-          logWarning(s"Attempted to decommission unknown executor $executorId.")
-          return false
-      }
-    }
-    logInfo(s"Asked executor $executorId to decommission.")
-
-    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
-      try {
-        logInfo(s"Asking block manager corresponding to executor $executorId to decommission.")
-        scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))

Review comment:
       One of the differences with the old design was that the 'Clearing of the block managers from the block manager master and the actual message to the decommission block manager' happened in one place (in scheduler.sc.env.blockManager.master.decommissionBlockManagers). 
   
   Whereas now these two things are split up: The scheduler.sc.env.blockManager.master.decommissionBlockManagers no longer sends the message, just updates its in memory state.
   
   Is there a problem with this difference ? Is some race opened up or closed ?

##########
File path: core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
##########
@@ -94,8 +94,9 @@ private[spark] trait ExecutorAllocationClient {
    * @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)],

Review comment:
       I keep messing this up myself, but is this indentation style change intentional ? If 2 spaces are okay then can we remain at that ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
##########
@@ -61,13 +61,28 @@ 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 are pending to be decommissioned.

Review comment:
       nit: which are pending to be decommissioned -> which should be decommissioned ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/master/Master.scala
##########
@@ -245,14 +245,14 @@ 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))

Review comment:
       We don't need to handle the MasterInStandby case anymore ? I guess all messages from worker to master should have this check, right ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
##########
@@ -768,11 +768,13 @@ private[deploy] class Worker(
     }
   }
 
-  private[deploy] def decommissionSelf(): Boolean = {
+  private[deploy] def decommissionSelf(fromMaster: Boolean): Boolean = {
     if (conf.get(config.DECOMMISSION_ENABLED)) {
       logDebug("Decommissioning self")
       decommissioned = true
-      sendToMaster(WorkerDecommission(workerId, self))
+      if (!fromMaster) {
+        sendToMaster(WorkerDecommissioned(workerId))

Review comment:
       I wonder if it is trite to add a comment like: "No need to notify the master if the decommission message already came from it"

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala
##########
@@ -95,8 +95,13 @@ 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 been
+  // used. It's used for the case where decommission is triggered at executor (e.g., K8S)
+  case class ExecutorDecommissioned(executorId: String)
+
+  // A message that sent from driver to executor to decommission that executor.
+  // It's used for Standalone's case yet, where decommission is triggered at Worker.

Review comment:
       remove 'yet'. Are you sure that this is when the decommission is triggered at the Worker ? I think it can also be used when the decommissioning is triggered at the MasterWebUI (not necessarily a SIGPWR at the Worker).

##########
File path: core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
##########
@@ -166,17 +166,6 @@ private[spark] class CoarseGrainedExecutorBackend(
       if (executor == null) {
         exitExecutor(1, "Received LaunchTask command but executor was null")
       } else {
-        if (decommissioned) {

Review comment:
       Are you sure that this case won't happen ? Or perhaps you mean that even if it does happen, it only lasts for a short duration: (until the Driver gets to know that the executor is decommissioned), that its not worth handling it ? If so, I agree with you. 
   
   But just want to confirm when can this case even happen: task launched on a decommissioned executor ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/master/Master.scala
##########
@@ -891,16 +891,13 @@ 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
   }
 
-  private def decommissionWorker(worker: WorkerInfo): Unit = {
+  private def decommissionWorker(worker: WorkerInfo, sentFromWorker: Boolean): Unit = {

Review comment:
       Would triggeredByWorker be a better name than sentFromWorker ?

##########
File path: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
##########
@@ -581,7 +581,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,
+          decommissionFromDriver = true)

Review comment:
       I think decommissionFromDriver = false means that decommission is triggered by a SIGPWR on the executor. 
   
   Whereas, decommissionFromDriver = true means that it might be triggered form either the Master or the Dynamic allocation manager. 
   
   If so, should we flip decommissionFromDriver and just call it decomTriggeredByExecutor ? 

##########
File path: core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
##########
@@ -768,11 +768,13 @@ private[deploy] class Worker(
     }
   }
 
-  private[deploy] def decommissionSelf(): Boolean = {
+  private[deploy] def decommissionSelf(fromMaster: Boolean): Boolean = {

Review comment:
       For consistency: triggeredByMaster instead of fromMaster ?

##########
File path: core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
##########
@@ -264,17 +253,21 @@ private[spark] class CoarseGrainedExecutorBackend(
     System.exit(code)
   }
 
-  private def decommissionSelf(): Boolean = {
+  private def decommissionSelf(fromDriver: Boolean): Boolean = {
     val msg = "Decommissioning self w/sync"
     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)) {

Review comment:
       I see, so is this the now ONLY place where block manager decommissioning is triggered ? Good.

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -272,10 +268,16 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
         removeWorker(workerId, host, message)
         context.reply(true)
 
-      case DecommissionExecutor(executorId, decommissionInfo) =>
-        logError(s"Received decommission executor message ${executorId}: ${decommissionInfo}.")
-        context.reply(decommissionExecutor(executorId, decommissionInfo,
-          adjustTargetNumExecutors = false))
+      case ExecutorDecommissioned(executorId) =>
+        logWarning(s"Received executor $executorId decommissioned message")
+        context.reply(
+          decommissionExecutor(
+            executorId,
+            ExecutorDecommissionInfo(s"Executor $executorId is decommissioned."),
+            adjustTargetNumExecutors = false,
+            // TODO: add a new type like `ExecutorDecommissionInfo` for the case where executor

Review comment:
       This is a good TODO comment, should we add this at the definition of the `decommissionExecutor` method instead of at the caller ?

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {

Review comment:
       nice ! I like the change to flatMap and withLock.

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -1809,7 +1811,7 @@ private[spark] class BlockManager(
     blocksToRemove.size
   }
 
-  def decommissionBlockManager(): Unit = synchronized {
+  private[spark] def decommissionSelf(): Unit = synchronized {

Review comment:
       Didn't follow the need for the name change.

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)
 
-  private def doDecommission(executorId: String,
-      decomInfo: ExecutorDecommissionInfo): Boolean = {
-
-    logInfo(s"Asking executor $executorId to decommissioning.")
-    scheduler.executorDecommission(executorId, decomInfo)
-    // Send decommission message to the executor (it could have originated on the executor
-    // but not necessarily).
-    CoarseGrainedSchedulerBackend.this.synchronized {
-      executorDataMap.get(executorId) match {
-        case Some(executorInfo) =>
-          executorInfo.executorEndpoint.send(DecommissionSelf)
-        case None =>
-          // Ignoring the executor since it is not registered.
-          logWarning(s"Attempted to decommission unknown executor $executorId.")
-          return false
-      }
-    }
-    logInfo(s"Asked executor $executorId to decommission.")
-
-    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
-      try {
-        logInfo(s"Asking block manager corresponding to executor $executorId to decommission.")
-        scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))
-      } catch {
-        case e: Exception =>
-          logError("Unexpected error during block manager " +
-            s"decommissioning for executor $executorId: ${e.toString}", e)
-          return false
+    if (decommissionFromDriver) {
+      CoarseGrainedSchedulerBackend.this.synchronized {

Review comment:
       Why the lock here ? It wasn't locked before (I think).

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)
 
-  private def doDecommission(executorId: String,
-      decomInfo: ExecutorDecommissionInfo): Boolean = {
-
-    logInfo(s"Asking executor $executorId to decommissioning.")
-    scheduler.executorDecommission(executorId, decomInfo)
-    // Send decommission message to the executor (it could have originated on the executor
-    // but not necessarily).
-    CoarseGrainedSchedulerBackend.this.synchronized {
-      executorDataMap.get(executorId) match {
-        case Some(executorInfo) =>
-          executorInfo.executorEndpoint.send(DecommissionSelf)
-        case None =>
-          // Ignoring the executor since it is not registered.
-          logWarning(s"Attempted to decommission unknown executor $executorId.")
-          return false
-      }
-    }
-    logInfo(s"Asked executor $executorId to decommission.")
-
-    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
-      try {
-        logInfo(s"Asking block manager corresponding to executor $executorId to decommission.")
-        scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))
-      } catch {
-        case e: Exception =>
-          logError("Unexpected error during block manager " +
-            s"decommissioning for executor $executorId: ${e.toString}", e)
-          return false
+    if (decommissionFromDriver) {
+      CoarseGrainedSchedulerBackend.this.synchronized {
+        executorsToDecommission.foreach { executorId =>
+          logInfo(s"Asking executor $executorId to decommissioning.")
+          executorDataMap(executorId).executorEndpoint.send(DecommissionExecutor)

Review comment:
        executorDataMap(executorId) might be None: if an executor was removed b/w the end of withLock on L485 to L500. The original code handled that. Should we handle it to ? (We don't have to log anything, just not crash the driver).

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)

Review comment:
       Should this check be guarded by `conf.get(STORAGE_DECOMMISSION_ENABLED)` ? Also should this only be done in case of decommissionFromDriver == true ?
   

##########
File path: core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala
##########
@@ -31,7 +31,7 @@ import org.apache.spark.util.{RpcUtils, SerializableBuffer, ThreadUtils}
 class WorkerDecommissionSuite extends SparkFunSuite with LocalSparkContext {
 
   override def beforeEach(): Unit = {
-    val conf = new SparkConf().setAppName("test").setMaster("local")

Review comment:
       Is master = "local" the default ? And that's why the .setMaster("local") is removed ?

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)
 
-  private def doDecommission(executorId: String,
-      decomInfo: ExecutorDecommissionInfo): Boolean = {
-
-    logInfo(s"Asking executor $executorId to decommissioning.")
-    scheduler.executorDecommission(executorId, decomInfo)
-    // Send decommission message to the executor (it could have originated on the executor
-    // but not necessarily).
-    CoarseGrainedSchedulerBackend.this.synchronized {
-      executorDataMap.get(executorId) match {
-        case Some(executorInfo) =>
-          executorInfo.executorEndpoint.send(DecommissionSelf)
-        case None =>
-          // Ignoring the executor since it is not registered.
-          logWarning(s"Attempted to decommission unknown executor $executorId.")
-          return false
-      }
-    }
-    logInfo(s"Asked executor $executorId to decommission.")
-
-    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
-      try {
-        logInfo(s"Asking block manager corresponding to executor $executorId to decommission.")
-        scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))

Review comment:
       One of the differences with the old design was that the 'Clearing of the block managers from the block manager master and the actual message to the decommission block manager' happened in one place (in scheduler.sc.env.blockManager.master.decommissionBlockManagers). 
   
   Whereas now these two things are split up: The scheduler.sc.env.blockManager.master.decommissionBlockManagers no longer sends the message, just updates its in memory state.
   
   Is there a problem with this difference ? Is some race opened up or closed ?

##########
File path: core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
##########
@@ -94,8 +94,9 @@ private[spark] trait ExecutorAllocationClient {
    * @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)],

Review comment:
       I keep messing this up myself, but is this indentation style change intentional ? If 2 spaces are okay then can we remain at that ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
##########
@@ -61,13 +61,28 @@ 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 are pending to be decommissioned.

Review comment:
       nit: which are pending to be decommissioned -> which should be decommissioned ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/master/Master.scala
##########
@@ -245,14 +245,14 @@ 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))

Review comment:
       We don't need to handle the MasterInStandby case anymore ? I guess all messages from worker to master should have this check, right ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
##########
@@ -768,11 +768,13 @@ private[deploy] class Worker(
     }
   }
 
-  private[deploy] def decommissionSelf(): Boolean = {
+  private[deploy] def decommissionSelf(fromMaster: Boolean): Boolean = {
     if (conf.get(config.DECOMMISSION_ENABLED)) {
       logDebug("Decommissioning self")
       decommissioned = true
-      sendToMaster(WorkerDecommission(workerId, self))
+      if (!fromMaster) {
+        sendToMaster(WorkerDecommissioned(workerId))

Review comment:
       I wonder if it is trite to add a comment like: "No need to notify the master if the decommission message already came from it"

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala
##########
@@ -95,8 +95,13 @@ 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 been
+  // used. It's used for the case where decommission is triggered at executor (e.g., K8S)
+  case class ExecutorDecommissioned(executorId: String)
+
+  // A message that sent from driver to executor to decommission that executor.
+  // It's used for Standalone's case yet, where decommission is triggered at Worker.

Review comment:
       remove 'yet'. Are you sure that this is when the decommission is triggered at the Worker ? I think it can also be used when the decommissioning is triggered at the MasterWebUI (not necessarily a SIGPWR at the Worker).

##########
File path: core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
##########
@@ -166,17 +166,6 @@ private[spark] class CoarseGrainedExecutorBackend(
       if (executor == null) {
         exitExecutor(1, "Received LaunchTask command but executor was null")
       } else {
-        if (decommissioned) {

Review comment:
       Are you sure that this case won't happen ? Or perhaps you mean that even if it does happen, it only lasts for a short duration: (until the Driver gets to know that the executor is decommissioned), that its not worth handling it ? If so, I agree with you. 
   
   But just want to confirm when can this case even happen: task launched on a decommissioned executor ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/master/Master.scala
##########
@@ -891,16 +891,13 @@ 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
   }
 
-  private def decommissionWorker(worker: WorkerInfo): Unit = {
+  private def decommissionWorker(worker: WorkerInfo, sentFromWorker: Boolean): Unit = {

Review comment:
       Would triggeredByWorker be a better name than sentFromWorker ?

##########
File path: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
##########
@@ -581,7 +581,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,
+          decommissionFromDriver = true)

Review comment:
       I think decommissionFromDriver = false means that decommission is triggered by a SIGPWR on the executor. 
   
   Whereas, decommissionFromDriver = true means that it might be triggered form either the Master or the Dynamic allocation manager. 
   
   If so, should we flip decommissionFromDriver and just call it decomTriggeredByExecutor ? 

##########
File path: core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
##########
@@ -768,11 +768,13 @@ private[deploy] class Worker(
     }
   }
 
-  private[deploy] def decommissionSelf(): Boolean = {
+  private[deploy] def decommissionSelf(fromMaster: Boolean): Boolean = {

Review comment:
       For consistency: triggeredByMaster instead of fromMaster ?

##########
File path: core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
##########
@@ -264,17 +253,21 @@ private[spark] class CoarseGrainedExecutorBackend(
     System.exit(code)
   }
 
-  private def decommissionSelf(): Boolean = {
+  private def decommissionSelf(fromDriver: Boolean): Boolean = {
     val msg = "Decommissioning self w/sync"
     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)) {

Review comment:
       I see, so is this the now ONLY place where block manager decommissioning is triggered ? Good.

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -272,10 +268,16 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
         removeWorker(workerId, host, message)
         context.reply(true)
 
-      case DecommissionExecutor(executorId, decommissionInfo) =>
-        logError(s"Received decommission executor message ${executorId}: ${decommissionInfo}.")
-        context.reply(decommissionExecutor(executorId, decommissionInfo,
-          adjustTargetNumExecutors = false))
+      case ExecutorDecommissioned(executorId) =>
+        logWarning(s"Received executor $executorId decommissioned message")
+        context.reply(
+          decommissionExecutor(
+            executorId,
+            ExecutorDecommissionInfo(s"Executor $executorId is decommissioned."),
+            adjustTargetNumExecutors = false,
+            // TODO: add a new type like `ExecutorDecommissionInfo` for the case where executor

Review comment:
       This is a good TODO comment, should we add this at the definition of the `decommissionExecutor` method instead of at the caller ?

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {

Review comment:
       nice ! I like the change to flatMap and withLock.

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -1809,7 +1811,7 @@ private[spark] class BlockManager(
     blocksToRemove.size
   }
 
-  def decommissionBlockManager(): Unit = synchronized {
+  private[spark] def decommissionSelf(): Unit = synchronized {

Review comment:
       Didn't follow the need for the name change.

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)
 
-  private def doDecommission(executorId: String,
-      decomInfo: ExecutorDecommissionInfo): Boolean = {
-
-    logInfo(s"Asking executor $executorId to decommissioning.")
-    scheduler.executorDecommission(executorId, decomInfo)
-    // Send decommission message to the executor (it could have originated on the executor
-    // but not necessarily).
-    CoarseGrainedSchedulerBackend.this.synchronized {
-      executorDataMap.get(executorId) match {
-        case Some(executorInfo) =>
-          executorInfo.executorEndpoint.send(DecommissionSelf)
-        case None =>
-          // Ignoring the executor since it is not registered.
-          logWarning(s"Attempted to decommission unknown executor $executorId.")
-          return false
-      }
-    }
-    logInfo(s"Asked executor $executorId to decommission.")
-
-    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
-      try {
-        logInfo(s"Asking block manager corresponding to executor $executorId to decommission.")
-        scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))
-      } catch {
-        case e: Exception =>
-          logError("Unexpected error during block manager " +
-            s"decommissioning for executor $executorId: ${e.toString}", e)
-          return false
+    if (decommissionFromDriver) {
+      CoarseGrainedSchedulerBackend.this.synchronized {

Review comment:
       Why the lock here ? It wasn't locked before (I think).

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)
 
-  private def doDecommission(executorId: String,
-      decomInfo: ExecutorDecommissionInfo): Boolean = {
-
-    logInfo(s"Asking executor $executorId to decommissioning.")
-    scheduler.executorDecommission(executorId, decomInfo)
-    // Send decommission message to the executor (it could have originated on the executor
-    // but not necessarily).
-    CoarseGrainedSchedulerBackend.this.synchronized {
-      executorDataMap.get(executorId) match {
-        case Some(executorInfo) =>
-          executorInfo.executorEndpoint.send(DecommissionSelf)
-        case None =>
-          // Ignoring the executor since it is not registered.
-          logWarning(s"Attempted to decommission unknown executor $executorId.")
-          return false
-      }
-    }
-    logInfo(s"Asked executor $executorId to decommission.")
-
-    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
-      try {
-        logInfo(s"Asking block manager corresponding to executor $executorId to decommission.")
-        scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))
-      } catch {
-        case e: Exception =>
-          logError("Unexpected error during block manager " +
-            s"decommissioning for executor $executorId: ${e.toString}", e)
-          return false
+    if (decommissionFromDriver) {
+      CoarseGrainedSchedulerBackend.this.synchronized {
+        executorsToDecommission.foreach { executorId =>
+          logInfo(s"Asking executor $executorId to decommissioning.")
+          executorDataMap(executorId).executorEndpoint.send(DecommissionExecutor)

Review comment:
        executorDataMap(executorId) might be None: if an executor was removed b/w the end of withLock on L485 to L500. The original code handled that. Should we handle it to ? (We don't have to log anything, just not crash the driver).

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)

Review comment:
       Should this check be guarded by `conf.get(STORAGE_DECOMMISSION_ENABLED)` ? Also should this only be done in case of decommissionFromDriver == true ?
   

##########
File path: core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala
##########
@@ -31,7 +31,7 @@ import org.apache.spark.util.{RpcUtils, SerializableBuffer, ThreadUtils}
 class WorkerDecommissionSuite extends SparkFunSuite with LocalSparkContext {
 
   override def beforeEach(): Unit = {
-    val conf = new SparkConf().setAppName("test").setMaster("local")

Review comment:
       Is master = "local" the default ? And that's why the .setMaster("local") is removed ?

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)
 
-  private def doDecommission(executorId: String,
-      decomInfo: ExecutorDecommissionInfo): Boolean = {
-
-    logInfo(s"Asking executor $executorId to decommissioning.")
-    scheduler.executorDecommission(executorId, decomInfo)
-    // Send decommission message to the executor (it could have originated on the executor
-    // but not necessarily).
-    CoarseGrainedSchedulerBackend.this.synchronized {
-      executorDataMap.get(executorId) match {
-        case Some(executorInfo) =>
-          executorInfo.executorEndpoint.send(DecommissionSelf)
-        case None =>
-          // Ignoring the executor since it is not registered.
-          logWarning(s"Attempted to decommission unknown executor $executorId.")
-          return false
-      }
-    }
-    logInfo(s"Asked executor $executorId to decommission.")
-
-    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
-      try {
-        logInfo(s"Asking block manager corresponding to executor $executorId to decommission.")
-        scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))

Review comment:
       One of the differences with the old design was that the 'Clearing of the block managers from the block manager master and the actual message to the decommission block manager' happened in one place (in scheduler.sc.env.blockManager.master.decommissionBlockManagers). 
   
   Whereas now these two things are split up: The scheduler.sc.env.blockManager.master.decommissionBlockManagers no longer sends the message, just updates its in memory state.
   
   Is there a problem with this difference ? Is some race opened up or closed ?

##########
File path: core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
##########
@@ -94,8 +94,9 @@ private[spark] trait ExecutorAllocationClient {
    * @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)],

Review comment:
       I keep messing this up myself, but is this indentation style change intentional ? If 2 spaces are okay then can we remain at that ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
##########
@@ -61,13 +61,28 @@ 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 are pending to be decommissioned.

Review comment:
       nit: which are pending to be decommissioned -> which should be decommissioned ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/master/Master.scala
##########
@@ -245,14 +245,14 @@ 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))

Review comment:
       We don't need to handle the MasterInStandby case anymore ? I guess all messages from worker to master should have this check, right ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
##########
@@ -768,11 +768,13 @@ private[deploy] class Worker(
     }
   }
 
-  private[deploy] def decommissionSelf(): Boolean = {
+  private[deploy] def decommissionSelf(fromMaster: Boolean): Boolean = {
     if (conf.get(config.DECOMMISSION_ENABLED)) {
       logDebug("Decommissioning self")
       decommissioned = true
-      sendToMaster(WorkerDecommission(workerId, self))
+      if (!fromMaster) {
+        sendToMaster(WorkerDecommissioned(workerId))

Review comment:
       I wonder if it is trite to add a comment like: "No need to notify the master if the decommission message already came from it"

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala
##########
@@ -95,8 +95,13 @@ 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 been
+  // used. It's used for the case where decommission is triggered at executor (e.g., K8S)
+  case class ExecutorDecommissioned(executorId: String)
+
+  // A message that sent from driver to executor to decommission that executor.
+  // It's used for Standalone's case yet, where decommission is triggered at Worker.

Review comment:
       remove 'yet'. Are you sure that this is when the decommission is triggered at the Worker ? I think it can also be used when the decommissioning is triggered at the MasterWebUI (not necessarily a SIGPWR at the Worker).

##########
File path: core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
##########
@@ -166,17 +166,6 @@ private[spark] class CoarseGrainedExecutorBackend(
       if (executor == null) {
         exitExecutor(1, "Received LaunchTask command but executor was null")
       } else {
-        if (decommissioned) {

Review comment:
       Are you sure that this case won't happen ? Or perhaps you mean that even if it does happen, it only lasts for a short duration: (until the Driver gets to know that the executor is decommissioned), that its not worth handling it ? If so, I agree with you. 
   
   But just want to confirm when can this case even happen: task launched on a decommissioned executor ?

##########
File path: core/src/main/scala/org/apache/spark/deploy/master/Master.scala
##########
@@ -891,16 +891,13 @@ 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
   }
 
-  private def decommissionWorker(worker: WorkerInfo): Unit = {
+  private def decommissionWorker(worker: WorkerInfo, sentFromWorker: Boolean): Unit = {

Review comment:
       Would triggeredByWorker be a better name than sentFromWorker ?

##########
File path: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
##########
@@ -581,7 +581,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,
+          decommissionFromDriver = true)

Review comment:
       I think decommissionFromDriver = false means that decommission is triggered by a SIGPWR on the executor. 
   
   Whereas, decommissionFromDriver = true means that it might be triggered form either the Master or the Dynamic allocation manager. 
   
   If so, should we flip decommissionFromDriver and just call it decomTriggeredByExecutor ? 

##########
File path: core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
##########
@@ -768,11 +768,13 @@ private[deploy] class Worker(
     }
   }
 
-  private[deploy] def decommissionSelf(): Boolean = {
+  private[deploy] def decommissionSelf(fromMaster: Boolean): Boolean = {

Review comment:
       For consistency: triggeredByMaster instead of fromMaster ?

##########
File path: core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
##########
@@ -264,17 +253,21 @@ private[spark] class CoarseGrainedExecutorBackend(
     System.exit(code)
   }
 
-  private def decommissionSelf(): Boolean = {
+  private def decommissionSelf(fromDriver: Boolean): Boolean = {
     val msg = "Decommissioning self w/sync"
     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)) {

Review comment:
       I see, so is this the now ONLY place where block manager decommissioning is triggered ? Good.

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -272,10 +268,16 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
         removeWorker(workerId, host, message)
         context.reply(true)
 
-      case DecommissionExecutor(executorId, decommissionInfo) =>
-        logError(s"Received decommission executor message ${executorId}: ${decommissionInfo}.")
-        context.reply(decommissionExecutor(executorId, decommissionInfo,
-          adjustTargetNumExecutors = false))
+      case ExecutorDecommissioned(executorId) =>
+        logWarning(s"Received executor $executorId decommissioned message")
+        context.reply(
+          decommissionExecutor(
+            executorId,
+            ExecutorDecommissionInfo(s"Executor $executorId is decommissioned."),
+            adjustTargetNumExecutors = false,
+            // TODO: add a new type like `ExecutorDecommissionInfo` for the case where executor

Review comment:
       This is a good TODO comment, should we add this at the definition of the `decommissionExecutor` method instead of at the caller ?

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {

Review comment:
       nice ! I like the change to flatMap and withLock.

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -1809,7 +1811,7 @@ private[spark] class BlockManager(
     blocksToRemove.size
   }
 
-  def decommissionBlockManager(): Unit = synchronized {
+  private[spark] def decommissionSelf(): Unit = synchronized {

Review comment:
       Didn't follow the need for the name change.

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)
 
-  private def doDecommission(executorId: String,
-      decomInfo: ExecutorDecommissionInfo): Boolean = {
-
-    logInfo(s"Asking executor $executorId to decommissioning.")
-    scheduler.executorDecommission(executorId, decomInfo)
-    // Send decommission message to the executor (it could have originated on the executor
-    // but not necessarily).
-    CoarseGrainedSchedulerBackend.this.synchronized {
-      executorDataMap.get(executorId) match {
-        case Some(executorInfo) =>
-          executorInfo.executorEndpoint.send(DecommissionSelf)
-        case None =>
-          // Ignoring the executor since it is not registered.
-          logWarning(s"Attempted to decommission unknown executor $executorId.")
-          return false
-      }
-    }
-    logInfo(s"Asked executor $executorId to decommission.")
-
-    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
-      try {
-        logInfo(s"Asking block manager corresponding to executor $executorId to decommission.")
-        scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))
-      } catch {
-        case e: Exception =>
-          logError("Unexpected error during block manager " +
-            s"decommissioning for executor $executorId: ${e.toString}", e)
-          return false
+    if (decommissionFromDriver) {
+      CoarseGrainedSchedulerBackend.this.synchronized {

Review comment:
       Why the lock here ? It wasn't locked before (I think).

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)
 
-  private def doDecommission(executorId: String,
-      decomInfo: ExecutorDecommissionInfo): Boolean = {
-
-    logInfo(s"Asking executor $executorId to decommissioning.")
-    scheduler.executorDecommission(executorId, decomInfo)
-    // Send decommission message to the executor (it could have originated on the executor
-    // but not necessarily).
-    CoarseGrainedSchedulerBackend.this.synchronized {
-      executorDataMap.get(executorId) match {
-        case Some(executorInfo) =>
-          executorInfo.executorEndpoint.send(DecommissionSelf)
-        case None =>
-          // Ignoring the executor since it is not registered.
-          logWarning(s"Attempted to decommission unknown executor $executorId.")
-          return false
-      }
-    }
-    logInfo(s"Asked executor $executorId to decommission.")
-
-    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
-      try {
-        logInfo(s"Asking block manager corresponding to executor $executorId to decommission.")
-        scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))
-      } catch {
-        case e: Exception =>
-          logError("Unexpected error during block manager " +
-            s"decommissioning for executor $executorId: ${e.toString}", e)
-          return false
+    if (decommissionFromDriver) {
+      CoarseGrainedSchedulerBackend.this.synchronized {
+        executorsToDecommission.foreach { executorId =>
+          logInfo(s"Asking executor $executorId to decommissioning.")
+          executorDataMap(executorId).executorEndpoint.send(DecommissionExecutor)

Review comment:
        executorDataMap(executorId) might be None: if an executor was removed b/w the end of withLock on L485 to L500. The original code handled that. Should we handle it to ? (We don't have to log anything, just not crash the driver).

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)

Review comment:
       Should this check be guarded by `conf.get(STORAGE_DECOMMISSION_ENABLED)` ? Also should this only be done in case of decommissionFromDriver == true ?
   

##########
File path: core/src/test/scala/org/apache/spark/scheduler/WorkerDecommissionSuite.scala
##########
@@ -31,7 +31,7 @@ import org.apache.spark.util.{RpcUtils, SerializableBuffer, ThreadUtils}
 class WorkerDecommissionSuite extends SparkFunSuite with LocalSparkContext {
 
   override def beforeEach(): Unit = {
-    val conf = new SparkConf().setAppName("test").setMaster("local")

Review comment:
       Is master = "local" the default ? And that's why the .setMaster("local") is removed ?

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)
 
-  private def doDecommission(executorId: String,
-      decomInfo: ExecutorDecommissionInfo): Boolean = {
-
-    logInfo(s"Asking executor $executorId to decommissioning.")
-    scheduler.executorDecommission(executorId, decomInfo)
-    // Send decommission message to the executor (it could have originated on the executor
-    // but not necessarily).
-    CoarseGrainedSchedulerBackend.this.synchronized {
-      executorDataMap.get(executorId) match {
-        case Some(executorInfo) =>
-          executorInfo.executorEndpoint.send(DecommissionSelf)
-        case None =>
-          // Ignoring the executor since it is not registered.
-          logWarning(s"Attempted to decommission unknown executor $executorId.")
-          return false
-      }
-    }
-    logInfo(s"Asked executor $executorId to decommission.")
-
-    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
-      try {
-        logInfo(s"Asking block manager corresponding to executor $executorId to decommission.")
-        scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))

Review comment:
       One of the differences with the old design was that the 'Clearing of the block managers from the block manager master and the actual message to the decommission block manager' happened in one place (in scheduler.sc.env.blockManager.master.decommissionBlockManagers). 
   
   Whereas now these two things are split up: The scheduler.sc.env.blockManager.master.decommissionBlockManagers no longer sends the message, just updates its in memory state.
   
   Is there a problem with this difference ? Is some race opened up or closed ?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r488343168



##########
File path: core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
##########
@@ -79,12 +79,23 @@ 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.send(DecommissionExecutor)

Review comment:
       I'm ok with adding a new mesage.
   
   WDYT? @cloud-fan 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-693216761






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29722: [SPARK-32850][CORE] Simply the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-690926561


   **[Test build #128559 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128559/testReport)** for PR 29722 at commit [`7efad16`](https://github.com/apache/spark/commit/7efad16a18c5174d25bf52e4cd754d2761091571).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692049328


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/128645/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r487853452



##########
File path: core/src/main/scala/org/apache/spark/deploy/master/Master.scala
##########
@@ -891,16 +896,13 @@ 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
   }
 
-  private def decommissionWorker(worker: WorkerInfo): Unit = {
+  private def decommissionWorker(worker: WorkerInfo, triggeredByWorker: Boolean): Unit = {

Review comment:
       There is only one place calling this method with `triggeredByWorker=false`. I think we can keep this method unchanged, and in that caller side:
   ```
   case DecommissionWorkers(ids) =>
     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, triggeredByWorker = false))
         // Also send a message to the worker node to notify.
         worker.endpoint.send(DecommissionWorker)
       }
     )
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #29722: [SPARK-32850][CORE] Simply the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-690851353


   **[Test build #128548 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128548/testReport)** for PR 29722 at commit [`3bca7c0`](https://github.com/apache/spark/commit/3bca7c001ed0fad7437b566e9c59132209755ee4).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-693216726


   **[Test build #128748 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128748/testReport)** for PR 29722 at commit [`290d2c0`](https://github.com/apache/spark/commit/290d2c08a1adfce3c0c4afe1cb8c9e214b25ea3d).
    * This patch **fails due to an unknown error code, -9**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692671219


   **[Test build #128709 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128709/testReport)** for PR 29722 at commit [`f64063e`](https://github.com/apache/spark/commit/f64063ed7f3ac2851ddfbdf92b4961fbd1a192c6).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692064880






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-691986108






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29722: [SPARK-32850][CORE] Simply the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-690851701






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-691992650






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692446530






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simply the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-690851701






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29722: [SPARK-32850][CORE] Simply the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-690851701






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-693214454






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r487628196



##########
File path: core/src/main/scala/org/apache/spark/deploy/master/Master.scala
##########
@@ -245,14 +245,14 @@ 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))

Review comment:
       Good catch!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-695907729


   I didn't realize that the k8s test is in docker and is out of the PR builder. This commit has been reverted. @Ngone51 please resubmit and fix the k8s test.
   
   We should probably improve test coverage for k8s decommission as well.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29722: [SPARK-32850][CORE] Simply the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-690851353


   **[Test build #128548 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128548/testReport)** for PR 29722 at commit [`3bca7c0`](https://github.com/apache/spark/commit/3bca7c001ed0fad7437b566e9c59132209755ee4).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on pull request #29722: [SPARK-32850][CORE] Simply the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-690851376


   @cloud-fan @agrawaldevesh @holdenk Please take a look, thanks!
    


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692672223






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692071552


   **[Test build #128654 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128654/testReport)** for PR 29722 at commit [`86b8b74`](https://github.com/apache/spark/commit/86b8b741b87d44d2da58590467e2cdedf0e4bf4b).
    * This patch **fails to build**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-691847911


   **[Test build #128620 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128620/testReport)** for PR 29722 at commit [`7940137`](https://github.com/apache/spark/commit/7940137fbb44ca6d7bd2ba6c190aebc7599fba71).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r487626850



##########
File path: core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
##########
@@ -94,8 +94,9 @@ private[spark] trait ExecutorAllocationClient {
    * @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)],

Review comment:
       It's intentional, please see: [For method declarations, use 4 space indentation for their parameters](https://github.com/databricks/scala-style-guide)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on pull request #29722: [SPARK-32850][CORE] Simply the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-690851376






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-693447292






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r487668029



##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)

Review comment:
       > Should this check be guarded by conf.get(STORAGE_DECOMMISSION_ENABLED) ? 
   
   Oops, I missed it.
   
   >  Also should this only be done in case of decommissionFromDriver == true ?
   
   No. BlockManagerMaster is the only place that has the global info of BlockManagers in the cluster. So wherever the decommission is triggered, we need to update the status(e.g., mark as decommissioned in this case) of the corresponding BlockManagers.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r488058077



##########
File path: core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
##########
@@ -79,12 +79,23 @@ 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.send(DecommissionExecutor)

Review comment:
       This is async, but I think it's safe here as the signal handler just needs to start decommissioning.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-693464684


   thanks, merging to master!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692455812


   **[Test build #128693 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128693/testReport)** for PR 29722 at commit [`f64063e`](https://github.com/apache/spark/commit/f64063ed7f3ac2851ddfbdf92b4961fbd1a192c6).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692575848


   **[Test build #128709 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128709/testReport)** for PR 29722 at commit [`f64063e`](https://github.com/apache/spark/commit/f64063ed7f3ac2851ddfbdf92b4961fbd1a192c6).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r488601401



##########
File path: core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
##########
@@ -79,12 +79,23 @@ 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.send(DecommissionExecutor)

Review comment:
       good idea! This is clearer and also makes the signal handler logic simpler.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692576651






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simply the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-690851701






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r489187225



##########
File path: core/src/main/scala/org/apache/spark/deploy/master/Master.scala
##########
@@ -245,15 +245,24 @@ 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)
+        idToWorker.get(id).foreach(w => decommissionWorker(w))
       }
 
+    case DecommissionWorkers(ids) =>
+      ids.foreach ( id =>

Review comment:
       ok, make sense.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692670633






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692512289


   **[Test build #128693 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128693/testReport)** for PR 29722 at commit [`f64063e`](https://github.com/apache/spark/commit/f64063ed7f3ac2851ddfbdf92b4961fbd1a192c6).
    * This patch **fails due to an unknown error code, -9**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-691985585


   **[Test build #128645 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128645/testReport)** for PR 29722 at commit [`8f6e279`](https://github.com/apache/spark/commit/8f6e279d5c4c0b2ac6acbe0a4ba88b82827cab61).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r487853452



##########
File path: core/src/main/scala/org/apache/spark/deploy/master/Master.scala
##########
@@ -891,16 +896,13 @@ 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
   }
 
-  private def decommissionWorker(worker: WorkerInfo): Unit = {
+  private def decommissionWorker(worker: WorkerInfo, triggeredByWorker: Boolean): Unit = {

Review comment:
       There is only one place calling this method with `triggeredByWorker=false`. I think we can keep this method unchanged, and in that caller side:
   ```
   case DecommissionWorkers(ids) =>
     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.
         worker.endpoint.send(DecommissionWorker)
       })
     )
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29722: [SPARK-32850][CORE] Simply the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-690923986






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r487633782



##########
File path: core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
##########
@@ -166,17 +166,6 @@ private[spark] class CoarseGrainedExecutorBackend(
       if (executor == null) {
         exitExecutor(1, "Received LaunchTask command but executor was null")
       } else {
-        if (decommissioned) {

Review comment:
       It does happen but it's meaningless to send a decommission notification to driver at this point. Because we assume that we've already sent the notification when `decommissioned=true`(if decommission is triggered at executor).
   
   It happens when `CoarseGrainedSchedulerBackend` handles `ReviveOffers` before `ExecutorDecommissioned`. (executor decommission can happen at any time so we can not guarantee the orders between `ReviveOffers` and `ExecutorDecommissioned`.)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-693319387


   **[Test build #128763 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128763/testReport)** for PR 29722 at commit [`290d2c0`](https://github.com/apache/spark/commit/290d2c08a1adfce3c0c4afe1cb8c9e214b25ea3d).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r487778265



##########
File path: core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
##########
@@ -70,7 +70,7 @@ 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.")(decommissionSelf(triggeredByMaster = false))

Review comment:
       shall we consistently use one name like `triggeredByWorker`?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692068264


   **[Test build #128654 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128654/testReport)** for PR 29722 at commit [`86b8b74`](https://github.com/apache/spark/commit/86b8b741b87d44d2da58590467e2cdedf0e4bf4b).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-693316416


   retest this please.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r487672016



##########
File path: core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
##########
@@ -264,17 +253,21 @@ private[spark] class CoarseGrainedExecutorBackend(
     System.exit(code)
   }
 
-  private def decommissionSelf(): Boolean = {
+  private def decommissionSelf(fromDriver: Boolean): Boolean = {
     val msg = "Decommissioning self w/sync"
     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)) {

Review comment:
       Yes.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692064880






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] agrawaldevesh commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
agrawaldevesh commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r488071027



##########
File path: core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
##########
@@ -79,12 +79,23 @@ 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.send(DecommissionExecutor)

Review comment:
       Okay I think I understand why ExecutorDecommissioning is only sent in this SIGPWR case: What do you think of introducing a new message like ExecutorSigPwrReceived and handle that in the event loop ? I am wary of doing all this work in the signal handling context. I would prefer it being done in the regular event loop.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692066868


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r489185786



##########
File path: core/src/main/scala/org/apache/spark/deploy/master/Master.scala
##########
@@ -245,15 +245,24 @@ 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)
+        idToWorker.get(id).foreach(w => decommissionWorker(w))
       }
 
+    case DecommissionWorkers(ids) =>
+      ids.foreach ( id =>

Review comment:
       or it will never happen because the request comes from master web UI?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-693214454






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r487855223



##########
File path: core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
##########
@@ -768,11 +768,14 @@ private[deploy] class Worker(
     }
   }
 
-  private[deploy] def decommissionSelf(): Boolean = {
+  private[deploy] def decommissionSelf(triggeredByWorker: Boolean): Boolean = {

Review comment:
       similarly, there is only one caller side using `triggeredByWorker=true`, we can keep this method unchanged and put the extra code to that caller side.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692576651






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r487832013



##########
File path: core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
##########
@@ -70,7 +70,7 @@ 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.")(decommissionSelf(triggeredByMaster = false))

Review comment:
       Ok, sounds good.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] holdenk commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
holdenk commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-695517007






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simply the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-690894738


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/128548/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29722: [SPARK-32850][CORE] Simply the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-691025552


   **[Test build #128559 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128559/testReport)** for PR 29722 at commit [`7efad16`](https://github.com/apache/spark/commit/7efad16a18c5174d25bf52e4cd754d2761091571).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simply the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-690851701






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29722: [SPARK-32850][CORE] Simply the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-690894733






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-691992650






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692572817


   retest this please.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692513177






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #29722: [SPARK-32850][CORE] Simply the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-690926561


   **[Test build #128559 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128559/testReport)** for PR 29722 at commit [`7efad16`](https://github.com/apache/spark/commit/7efad16a18c5174d25bf52e4cd754d2761091571).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692048486


   **[Test build #128645 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128645/testReport)** for PR 29722 at commit [`8f6e279`](https://github.com/apache/spark/commit/8f6e279d5c4c0b2ac6acbe0a4ba88b82827cab61).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692049323






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-691861666


   **[Test build #128620 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128620/testReport)** for PR 29722 at commit [`7940137`](https://github.com/apache/spark/commit/7940137fbb44ca6d7bd2ba6c190aebc7599fba71).
    * This patch **fails due to an unknown error code, -9**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r487675868



##########
File path: core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
##########
@@ -581,7 +581,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,
+          decommissionFromDriver = true)

Review comment:
       Let's be consistent with Master and Worker: `triggeredByExecutor`?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-691898112


   **[Test build #128633 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128633/testReport)** for PR 29722 at commit [`c922fb0`](https://github.com/apache/spark/commit/c922fb0421be63a23469745d187ee6c9b1755156).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692064134


   **[Test build #128653 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128653/testReport)** for PR 29722 at commit [`541a6e3`](https://github.com/apache/spark/commit/541a6e3d8c80d152e443d06b436fd6febdd2df13).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-693316696






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-691898112


   **[Test build #128633 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128633/testReport)** for PR 29722 at commit [`c922fb0`](https://github.com/apache/spark/commit/c922fb0421be63a23469745d187ee6c9b1755156).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-691861758






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r487853452



##########
File path: core/src/main/scala/org/apache/spark/deploy/master/Master.scala
##########
@@ -891,16 +896,13 @@ 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
   }
 
-  private def decommissionWorker(worker: WorkerInfo): Unit = {
+  private def decommissionWorker(worker: WorkerInfo, triggeredByWorker: Boolean): Unit = {

Review comment:
       There is only one place calling this method with `triggeredByWorker=false`. I think we can keep this method unchanged, and in that caller side:
   ```
   case DecommissionWorkers(ids) =>
     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.
         worker.endpoint.send(DecommissionWorker)
       }
     }
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r487669091



##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
##########
@@ -467,67 +469,44 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
    */
   override def decommissionExecutors(
       executorsAndDecomInfo: Array[(String, ExecutorDecommissionInfo)],
-      adjustTargetNumExecutors: Boolean): Seq[String] = {
-
-    val executorsToDecommission = executorsAndDecomInfo.filter { case (executorId, decomInfo) =>
-      CoarseGrainedSchedulerBackend.this.synchronized {
+      adjustTargetNumExecutors: Boolean,
+      decommissionFromDriver: Boolean): Seq[String] = {
+    val executorsToDecommission = withLock {
+      executorsAndDecomInfo.flatMap { case (executorId, decomInfo) =>
         // Only bother decommissioning executors which are alive.
         if (isExecutorActive(executorId)) {
+          scheduler.executorDecommission(executorId, decomInfo)
           executorsPendingDecommission(executorId) = decomInfo.workerHost
-          true
+          Some(executorId)
         } else {
-          false
+          None
         }
       }
     }
 
     // If we don't want to replace the executors we are decommissioning
     if (adjustTargetNumExecutors) {
-      adjustExecutors(executorsToDecommission.map(_._1))
+      adjustExecutors(executorsToDecommission)
     }
 
-    executorsToDecommission.filter { case (executorId, decomInfo) =>
-      doDecommission(executorId, decomInfo)
-    }.map(_._1)
-  }
-
+    // Mark those corresponding BlockManagers as decommissioned first before we sending
+    // decommission notification to executors. So, it's less likely to lead to the race
+    // condition where `getPeer` request from the decommissioned executor comes first
+    // before the BlockManagers are marked as decommissioned.
+    scheduler.sc.env.blockManager.master.decommissionBlockManagers(executorsToDecommission)
 
-  private def doDecommission(executorId: String,
-      decomInfo: ExecutorDecommissionInfo): Boolean = {
-
-    logInfo(s"Asking executor $executorId to decommissioning.")
-    scheduler.executorDecommission(executorId, decomInfo)
-    // Send decommission message to the executor (it could have originated on the executor
-    // but not necessarily).
-    CoarseGrainedSchedulerBackend.this.synchronized {
-      executorDataMap.get(executorId) match {
-        case Some(executorInfo) =>
-          executorInfo.executorEndpoint.send(DecommissionSelf)
-        case None =>
-          // Ignoring the executor since it is not registered.
-          logWarning(s"Attempted to decommission unknown executor $executorId.")
-          return false
-      }
-    }
-    logInfo(s"Asked executor $executorId to decommission.")
-
-    if (conf.get(STORAGE_DECOMMISSION_ENABLED)) {
-      try {
-        logInfo(s"Asking block manager corresponding to executor $executorId to decommission.")
-        scheduler.sc.env.blockManager.master.decommissionBlockManagers(Seq(executorId))
-      } catch {
-        case e: Exception =>
-          logError("Unexpected error during block manager " +
-            s"decommissioning for executor $executorId: ${e.toString}", e)
-          return false
+    if (decommissionFromDriver) {
+      CoarseGrainedSchedulerBackend.this.synchronized {
+        executorsToDecommission.foreach { executorId =>
+          logInfo(s"Asking executor $executorId to decommissioning.")
+          executorDataMap(executorId).executorEndpoint.send(DecommissionExecutor)

Review comment:
       Good point! I think we can add everything inside `withLock`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692095758


   **[Test build #128656 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128656/testReport)** for PR 29722 at commit [`d31f020`](https://github.com/apache/spark/commit/d31f0200c18d722c3357b85265d5c45a44801bf6).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r487928796



##########
File path: core/src/main/scala/org/apache/spark/deploy/master/Master.scala
##########
@@ -891,16 +896,13 @@ 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
   }
 
-  private def decommissionWorker(worker: WorkerInfo): Unit = {
+  private def decommissionWorker(worker: WorkerInfo, triggeredByWorker: Boolean): Unit = {

Review comment:
       Sounds good.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-693447292






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692758351


   **[Test build #128711 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128711/testReport)** for PR 29722 at commit [`9debec8`](https://github.com/apache/spark/commit/9debec82fb836ab334d93b18b36b4b7f2f9129cc).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `  case class DecommissionWorkers(ids: Seq[String]) extends DeployMessage`
     * `  case class ExecutorDecommissioning(executorId: String) extends CoarseGrainedClusterMessage`


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692575848


   **[Test build #128709 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/128709/testReport)** for PR 29722 at commit [`f64063e`](https://github.com/apache/spark/commit/f64063ed7f3ac2851ddfbdf92b4961fbd1a192c6).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692759802






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simply the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-690894733


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r489187271



##########
File path: core/src/main/scala/org/apache/spark/deploy/master/Master.scala
##########
@@ -245,15 +245,24 @@ 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)
+        idToWorker.get(id).foreach(w => decommissionWorker(w))
       }
 
+    case DecommissionWorkers(ids) =>
+      ids.foreach ( id =>

Review comment:
       ah i see




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-691848628






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #29722:
URL: https://github.com/apache/spark/pull/29722#discussion_r487699399



##########
File path: core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala
##########
@@ -61,13 +61,28 @@ 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])
+
+  /**
+   * 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 from Worker to Master to tell Master that the Worker has been
+   * decommissioned. It's used for the case where decommission is triggered at Worker.
+   *
    * @param id the worker id
-   * @param worker the worker endpoint ref
    */
-  case class WorkerDecommission(
-      id: String,
-      worker: RpcEndpointRef)
-    extends DeployMessage
+  case class WorkerDecommissioned(id: String, workerRef: RpcEndpointRef) extends DeployMessage

Review comment:
       how about `WorkerDecommissioning`? The worker is not decommissioned yet when sending this event.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #29722: [SPARK-32850][CORE] Simplify the RPC message flow of decommission

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #29722:
URL: https://github.com/apache/spark/pull/29722#issuecomment-692759802






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org