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 2022/08/05 00:44:41 UTC

[GitHub] [spark] kevin85421 opened a new pull request, #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

kevin85421 opened a new pull request, #37411:
URL: https://github.com/apache/spark/pull/37411

   ### What changes were proposed in this pull request?
   This PR aims to provide a method to lower the timeout. Our solution is to ask master for worker’s heartbeat when Driver does not receive heartbeat from executor for `TimeoutThreshold` seconds.
   
   ![Screen Shot 2022-08-01 at 6 10 56 PM](https://user-images.githubusercontent.com/20109646/182272331-2f972aa8-31c9-4c3e-8c88-b7cf2616fad8.png)
   
   In Databricks, driver and master processes are running on the master node. Executor and worker processes are running on the worker node but different JVMs. Therefore, the network connection between driver/executor and master/worker is equivalent because they are running on same physical nodes. 
   
   When Executor performs full GC, it cannot send any message during full GC. Next, Driver cannot receive heartbeat from Executor. Instead of removing the executor directly, driver will ask master for `workerLastHeartbeat`. Driver will determine whether it is network disconnection or other issues (e.g. GC) based on `workerLastHeartbeat`. If it is network disconnection, we will remove the executor. Otherwise, we will put the executor into a waitingList rather than expiring it immediately.
   
   * Result
   ![Screen Shot 2022-08-01 at 6 54 51 PM](https://user-images.githubusercontent.com/20109646/182275086-474d7458-f2a8-473c-adb2-c13f5c942ea1.png)
   
   ### Why are the changes needed?
   Currently, the driver’s HeartbeatReceiver will expire an executor if it does not receive any heartbeat from the executor for 120 seconds. However, 120 seconds is too long, but we will face other challenges when we try to lower the timeout threshold. To elaborate, when an executor is performing GC, it cannot reply any message.
   
   ![Screen Shot 2022-08-01 at 6 03 50 PM](https://user-images.githubusercontent.com/20109646/182269820-4802877d-a4e4-4d20-969d-6ece37ffdb55.png)
   
   We will use the above figure to explain why we cannot lower `TimeoutThreshold` to 60 seconds directly. When Executor performs full GC, it cannot send any message, including heartbeat. Next, driver will remove the executor because driver cannot receive heartbeat from Executor for 60 seconds. In other words, we cannot distinguish between GC and network disconnection.
   
   
   ### Does this PR introduce _any_ user-facing change?
   No
   
   
   ### How was this patch tested?
   ```
   build/sbt "core/testOnly *HeartbeatReceiverSuite"
   build/sbt "core/testOnly *MasterSuite"
   ```
   


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] mridulm commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r940864007


##########
core/src/main/scala/org/apache/spark/internal/config/package.scala:
##########
@@ -2398,4 +2398,11 @@ package object config {
       .version("3.3.0")
       .intConf
       .createWithDefault(5)
+
+  private[spark] val HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT =
+    ConfigBuilder("spark.driver.heartbeat.checkWorkerLastHeartbeat")
+      .internal()
+      .version("3.4.0")
+      .booleanConf
+      .createWithDefault(true)

Review Comment:
   Please update the relevant tests, so that they test with this being `true`/`false` as relevant - instead of a one-time CI/CD pass : so that future evolution of the code will continue to test this feature (with and without the flag as relevant)



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r941648441


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -77,17 +77,44 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
 
   private[spark] var scheduler: TaskScheduler = null
 
-  // executor ID -> timestamp of when the last heartbeat from this executor was received
+  /**
+   * [SPARK-39984]
+   * Please make sure the intersection between `executorLastSeen` and `waitingList` is an empty set.
+   * If the intersection is not empty, it is possible to never kill the executor until the executor
+   * recovers. When an executor is in both `executorLastSeen` and `waitingList`, the value of
+   * `workerLastHeartbeat` in waitingList may update if the worker sends heartbeats to master
+   * normally.
+   *
+   * `executorLastSeen`:
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from this executor was received
+   *
+   *  `waitingList`: executor ID -> WorkerLastHeartbeat
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from the worker was received
+   *
+   * when driver does not receive any heartbeat from an executor for `executorTimeoutMs` seconds,
+   * the driver will ask master for the last heartbeat from the worker which the executor is running
+   * on.
+   */
   private val executorLastSeen = new HashMap[String, Long]
+  private val waitingList = new HashMap[String, Long]
 
   private val executorTimeoutMs = sc.conf.get(
     config.STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT
-  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s"))
+  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_EXECUTOR_TIMEOUT)}s"))
 
   private val checkTimeoutIntervalMs = sc.conf.get(Network.NETWORK_TIMEOUT_INTERVAL)
 
   private val executorHeartbeatIntervalMs = sc.conf.get(config.EXECUTOR_HEARTBEAT_INTERVAL)
 
+  /**
+   * Currently, [SPARK-39984] is only for StandaloneSchedulerBackend.
+   */
+  private val checkWorkerLastHeartbeat =
+    sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT) &&
+      sc.schedulerBackend.isInstanceOf[StandaloneSchedulerBackend]

Review Comment:
   Heartbeat receiver will receive all of the messages (`ExecutorRegistered` / `ExecutorRemoved` / `Heartbeat`) only after SchedulerBackend is initialized. Take `ExecutorRegistered` as an example, scheduler backend has already been initialized in Step1, and used the message `SparkListenerExecutorAdded` to notify HeartbeatReceiver to register the new executor.
   
   * Step1: SchedulerBackend (CoarseGrainedSchedulerBackend / LocalSchedulerBackend)
      * `post(SparkListenerExecutorAdded)`
   * Step2: `doPostEvent` (SparkListenerBus) 
   * Step3: `onExecutorAdded` (HeartbeatReceiver)
   * Step4: `addExecutor` (HeartbeatReceiver)
   * Step5: `ExecutorRegistered` (HeartbeatReceiver)



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r963378869


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -77,17 +77,61 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
 
   private[spark] var scheduler: TaskScheduler = null
 
-  // executor ID -> timestamp of when the last heartbeat from this executor was received
+  /**
+   * [SPARK-39984]
+   * Please make sure the intersection between `executorLastSeen` and `executorExpiryCandidates` is
+   * an empty set. If the intersection is not empty, it is possible to never kill the executor until
+   * the executor recovers. When an executor is in both `executorLastSeen` and
+   * `executorExpiryCandidates`, the value of `workerLastHeartbeat` in `executorExpiryCandidates`
+   * may update if the worker sends heartbeats to master normally.
+   *
+   * `executorLastSeen`:
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from this executor was received
+   *
+   * `executorExpiryCandidates`: executor ID -> WorkerLastHeartbeat
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from the worker was received
+   *
+   * when driver does not receive any heartbeat from an executor for `executorTimeoutMs` seconds,
+   * the driver will ask master for the last heartbeat from the worker which the executor is running
+   * on.
+   */
   private val executorLastSeen = new HashMap[String, Long]
+  private val executorExpiryCandidates = new HashMap[String, Long]
 
   private val executorTimeoutMs = sc.conf.get(
     config.STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT
-  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s"))
+  ).getOrElse(
+    sc.conf.get(Network.NETWORK_EXECUTOR_TIMEOUT) match {
+      case Some(executorTimeout) => executorTimeout
+      case None => Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s")
+    }
+  )
 
   private val checkTimeoutIntervalMs = sc.conf.get(Network.NETWORK_TIMEOUT_INTERVAL)
 
   private val executorHeartbeatIntervalMs = sc.conf.get(config.EXECUTOR_HEARTBEAT_INTERVAL)
 
+  /**
+   * Currently, [SPARK-39984] is only for StandaloneSchedulerBackend.
+   *
+   * `checkWorkerLastHeartbeat`: A flag to enable two-phase executor timeout.
+   * `expiryCandidatesTimeout`: The timeout used for executorExpiryCandidates.
+   */
+  private val checkWorkerLastHeartbeat = {
+    val isEnabled = sc.conf.get(config.HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT)
+    if (isEnabled) logWarning(s"Keep `expiryCandidatesTimeout` larger than `HEARTBEAT_MILLIS` in" +
+      s"deploy/worker/Worker.scala to know whether master lost any heartbeat from the" +
+      s"worker or not.")
+    isEnabled
+  }
+
+  private val expiryCandidatesTimeout = checkWorkerLastHeartbeat match {

Review Comment:
   Updated https://github.com/apache/spark/pull/37411/commits/92629e30410d7ae9741457240c3f1a789f6b042b



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] github-actions[bot] closed pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
github-actions[bot] closed pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor
URL: https://github.com/apache/spark/pull/37411


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r946373236


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +222,131 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms",
+                causedByApp = !sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT))))
+
+          // LocalSchedulerBackend is used locally and only has one single executor
+          case _: LocalSchedulerBackend =>
+
+          case other => throw new UnsupportedOperationException(
+            s"Unknown scheduler backend: ${other.getClass}")
+        }
+      }
+    })
+  }
+
   private def expireDeadHosts(): Unit = {
+  /**
+   * [SC-105641]
+   * Originally, the driver’s HeartbeatReceiver will expire an executor if it does not receive any
+   * heartbeat from the executor for 120 seconds. However, 120 seconds is too long, but we will face
+   * other challenges when we try to lower the timeout threshold. To elaborate, when an executor is
+   * performing full GC, it cannot send/reply any message. Next paragraphs describe the solution to
+   * detect network disconnection between driver and executor in a short time.
+   *
+   * An executor is running on a worker but in different JVMs, and a driver is running on a master
+   * but in different JVMs. Hence, the network connection between driver/executor and master/worker
+   * is the same. Because executor and worker are running on different JVMs, worker can still send
+   * heartbeat to master when executor performs GC.
+   *
+   * For new Heartbeat Receiver, if driver does not receive any heartbeat from the executor for
+   * `executorTimeoutMs` (default: 60s) seconds, HeartbeatReceiver will send a request to master to
+   * ask for the latest heartbeat from the worker which the executor runs on `workerLastHeartbeat`.
+   * HeartbeatReceiver can determine whether the heartbeat loss is caused by network issues or other
+   * issues (e.g. GC). If the heartbeat loss is not caused by network issues, the HeartbeatReceiver
+   * will put the executor into a waitingList rather than expiring it immediately.
+   *
+   * [Note]: Definition of `network issues`
+   * Here, the definition `network issues` is the issues that related to network directly. If the
+   * network is connected, the issues do not included in `network issues`. For example, an
+   * executor's JVM is closed by a problematic task, so the JVM will notify driver that the socket
+   * is closed. If the network is connected, driver will receive the notification and trigger the
+   * function `onDisconnected`. This issue is not a `network issue` because the network is
+   * connected.
+   *
+   * [Warning 1]
+   * Worker will send heartbeats to Master every (conf.get(WORKER_TIMEOUT) * 1000 / 4) milliseconds.
+   * Check deploy/worker/Worker.scala for more details. This new mechanism design is based on the
+   * assumption: (executorTimeoutMs / 2) > (conf.get(WORKER_TIMEOUT) * 1000 / 4).
+   *
+   * [Warning 2]
+   * Not every deployment method schedules driver on master.
+   */
     logTrace("Checking for hosts with no recent heartbeats in HeartbeatReceiver.")
     val now = clock.getTimeMillis()
-    for ((executorId, lastSeenMs) <- executorLastSeen) {
-      if (now - lastSeenMs > executorTimeoutMs) {
-        logWarning(s"Removing executor $executorId with no recent heartbeats: " +
-          s"${now - lastSeenMs} ms exceeds timeout $executorTimeoutMs ms")
-        // Asynchronously kill the executor to avoid blocking the current thread
-        killExecutorThread.submit(new Runnable {
-          override def run(): Unit = Utils.tryLogNonFatalError {
-            // Note: we want to get an executor back after expiring this one,
-            // so do not simply call `sc.killExecutor` here (SPARK-8119)
-            sc.killAndReplaceExecutor(executorId)
-            // SPARK-27348: in case of the executors which are not gracefully shut down,
-            // we should remove lost executors from CoarseGrainedSchedulerBackend manually
-            // here to guarantee two things:
-            // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
-            //    a lost executor instead of waiting for disconnect message
-            // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
-            //    those executors to avoid app hang
-            sc.schedulerBackend match {
-              case backend: CoarseGrainedSchedulerBackend =>
-                backend.driverEndpoint.send(RemoveExecutor(executorId,
-                  ExecutorProcessLost(
-                    s"Executor heartbeat timed out after ${now - lastSeenMs} ms")))
-
-              // LocalSchedulerBackend is used locally and only has one single executor
-              case _: LocalSchedulerBackend =>
-
-              case other => throw new UnsupportedOperationException(
-                s"Unknown scheduler backend: ${other.getClass}")
-            }
+    if (!sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT)) {
+      for ((executorId, lastSeenMs) <- executorLastSeen) {
+        if (now - lastSeenMs > executorTimeoutMs) {
+          killExecutor(executorId, now - lastSeenMs)
+          waitingList.remove(executorId)
+          executorLastSeen.remove(executorId)
+        }
+      }
+    } else {
+      for ((executorId, workerLastHeartbeat) <- waitingList) {
+        if (now - workerLastHeartbeat > executorTimeoutMs / 2) {
+          killExecutor(executorId, now - workerLastHeartbeat)
+          waitingList.remove(executorId)
+          executorLastSeen.remove(executorId)
+        }
+      }
+
+      val buf = new ArrayBuffer[String]()
+      for ((executorId, lastSeenMs) <- executorLastSeen) {
+        if (now - lastSeenMs > executorTimeoutMs) {
+          sc.schedulerBackend match {
+            case _: StandaloneSchedulerBackend =>
+              buf += executorId
+            case _ =>
+              killExecutor(executorId, now - lastSeenMs)
+              waitingList.remove(executorId)
+              executorLastSeen.remove(executorId)
+          }
+        }
+      }
+
+      sc.schedulerBackend match {
+        case backend: StandaloneSchedulerBackend =>
+          backend.client.workerLastHeartbeat(sc.applicationId, buf) match {
+            case Some(workerLastHeartbeats) =>
+              for ((executorId, workerLastHeartbeat) <- buf zip workerLastHeartbeats) {
+                if (now - workerLastHeartbeat > executorTimeoutMs / 2) {
+                  val lastSeenMs = executorLastSeen.get(executorId).get
+                  killExecutor(executorId, now - lastSeenMs)
+                  waitingList.remove(executorId)
+                } else {
+                  waitingList(executorId) = workerLastHeartbeat
+                }
+                executorLastSeen.remove(executorId)
+              }
+            case None =>
+              for (executorId <- buf) {
+                val lastSeenMs = executorLastSeen.get(executorId).get
+                killExecutor(executorId, now - lastSeenMs)
+                executorLastSeen.remove(executorId)
+                waitingList.remove(executorId)
+              }

Review Comment:
   @Ngone51 Here is one of the discussions about the lack of scheduler backend.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r946374353


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +241,120 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            // TODO (SPARK-39984): Update causedByApp when we have a hanging task detector
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms")))
+          // LocalSchedulerBackend is used locally and only has one single executor
+          case _: LocalSchedulerBackend =>
+
+          case other => throw new UnsupportedOperationException(
+            s"Unknown scheduler backend: ${other.getClass}")
+        }
+      }
+    })
+  }
+
+  private def isStandalone(): Boolean = {
+    sc.schedulerBackend match {
+      case backend: StandaloneSchedulerBackend => true
+      case _ => false
+    }

Review Comment:
   Hi @Ngone51, thank you for your review! 
   
   Because `schedulerBackend` is possible to be None, I used pattern matching instead of `isInstanceOf`. @mridulm and I have discussed the issues above. I will tag you in these two discussion threads.
   
   Updated:
   I checked the syntax again. The value of `None.isInstanceOf[StandaloneSchedulerBackend]` is false instead of throwing a null pointer exception. You are correct. We can use `isInstanceOf` to replace pattern matching.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r942092334


##########
core/src/main/scala/org/apache/spark/internal/config/package.scala:
##########
@@ -2398,4 +2398,11 @@ package object config {
       .version("3.3.0")
       .intConf
       .createWithDefault(5)
+
+  private[spark] val HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT =
+    ConfigBuilder("spark.driver.heartbeat.checkWorkerLastHeartbeat")
+      .internal()
+      .version("3.4.0")
+      .booleanConf
+      .createWithDefault(true)

Review Comment:
   Updated. https://github.com/apache/spark/pull/37411/commits/bb5c2193962405f50242d7728a5cc6529d7eb229



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on PR #37411:
URL: https://github.com/apache/spark/pull/37411#issuecomment-1207165807

   Thank @mridulm for your recommendations! I will resolve these comments as soon as possible.


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r940753007


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +222,131 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms",
+                causedByApp = !sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT))))
+
+          // LocalSchedulerBackend is used locally and only has one single executor
+          case _: LocalSchedulerBackend =>
+
+          case other => throw new UnsupportedOperationException(
+            s"Unknown scheduler backend: ${other.getClass}")
+        }
+      }
+    })
+  }
+
   private def expireDeadHosts(): Unit = {
+  /**
+   * [SC-105641]
+   * Originally, the driver’s HeartbeatReceiver will expire an executor if it does not receive any
+   * heartbeat from the executor for 120 seconds. However, 120 seconds is too long, but we will face
+   * other challenges when we try to lower the timeout threshold. To elaborate, when an executor is
+   * performing full GC, it cannot send/reply any message. Next paragraphs describe the solution to
+   * detect network disconnection between driver and executor in a short time.
+   *
+   * An executor is running on a worker but in different JVMs, and a driver is running on a master
+   * but in different JVMs. Hence, the network connection between driver/executor and master/worker
+   * is the same. Because executor and worker are running on different JVMs, worker can still send
+   * heartbeat to master when executor performs GC.
+   *
+   * For new Heartbeat Receiver, if driver does not receive any heartbeat from the executor for
+   * `executorTimeoutMs` (default: 60s) seconds, HeartbeatReceiver will send a request to master to
+   * ask for the latest heartbeat from the worker which the executor runs on `workerLastHeartbeat`.
+   * HeartbeatReceiver can determine whether the heartbeat loss is caused by network issues or other
+   * issues (e.g. GC). If the heartbeat loss is not caused by network issues, the HeartbeatReceiver
+   * will put the executor into a waitingList rather than expiring it immediately.
+   *
+   * [Note]: Definition of `network issues`
+   * Here, the definition `network issues` is the issues that related to network directly. If the
+   * network is connected, the issues do not included in `network issues`. For example, an
+   * executor's JVM is closed by a problematic task, so the JVM will notify driver that the socket
+   * is closed. If the network is connected, driver will receive the notification and trigger the
+   * function `onDisconnected`. This issue is not a `network issue` because the network is
+   * connected.
+   *
+   * [Warning 1]
+   * Worker will send heartbeats to Master every (conf.get(WORKER_TIMEOUT) * 1000 / 4) milliseconds.
+   * Check deploy/worker/Worker.scala for more details. This new mechanism design is based on the
+   * assumption: (executorTimeoutMs / 2) > (conf.get(WORKER_TIMEOUT) * 1000 / 4).
+   *
+   * [Warning 2]
+   * Not every deployment method schedules driver on master.
+   */
     logTrace("Checking for hosts with no recent heartbeats in HeartbeatReceiver.")
     val now = clock.getTimeMillis()
-    for ((executorId, lastSeenMs) <- executorLastSeen) {
-      if (now - lastSeenMs > executorTimeoutMs) {
-        logWarning(s"Removing executor $executorId with no recent heartbeats: " +
-          s"${now - lastSeenMs} ms exceeds timeout $executorTimeoutMs ms")
-        // Asynchronously kill the executor to avoid blocking the current thread
-        killExecutorThread.submit(new Runnable {
-          override def run(): Unit = Utils.tryLogNonFatalError {
-            // Note: we want to get an executor back after expiring this one,
-            // so do not simply call `sc.killExecutor` here (SPARK-8119)
-            sc.killAndReplaceExecutor(executorId)
-            // SPARK-27348: in case of the executors which are not gracefully shut down,
-            // we should remove lost executors from CoarseGrainedSchedulerBackend manually
-            // here to guarantee two things:
-            // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
-            //    a lost executor instead of waiting for disconnect message
-            // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
-            //    those executors to avoid app hang
-            sc.schedulerBackend match {
-              case backend: CoarseGrainedSchedulerBackend =>
-                backend.driverEndpoint.send(RemoveExecutor(executorId,
-                  ExecutorProcessLost(
-                    s"Executor heartbeat timed out after ${now - lastSeenMs} ms")))
-
-              // LocalSchedulerBackend is used locally and only has one single executor
-              case _: LocalSchedulerBackend =>
-
-              case other => throw new UnsupportedOperationException(
-                s"Unknown scheduler backend: ${other.getClass}")
-            }
+    if (!sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT)) {
+      for ((executorId, lastSeenMs) <- executorLastSeen) {
+        if (now - lastSeenMs > executorTimeoutMs) {
+          killExecutor(executorId, now - lastSeenMs)
+          waitingList.remove(executorId)
+          executorLastSeen.remove(executorId)
+        }
+      }
+    } else {
+      for ((executorId, workerLastHeartbeat) <- waitingList) {
+        if (now - workerLastHeartbeat > executorTimeoutMs / 2) {
+          killExecutor(executorId, now - workerLastHeartbeat)
+          waitingList.remove(executorId)
+          executorLastSeen.remove(executorId)
+        }
+      }
+
+      val buf = new ArrayBuffer[String]()
+      for ((executorId, lastSeenMs) <- executorLastSeen) {
+        if (now - lastSeenMs > executorTimeoutMs) {
+          sc.schedulerBackend match {
+            case _: StandaloneSchedulerBackend =>
+              buf += executorId
+            case _ =>
+              killExecutor(executorId, now - lastSeenMs)
+              waitingList.remove(executorId)
+              executorLastSeen.remove(executorId)
+          }
+        }
+      }
+
+      sc.schedulerBackend match {
+        case backend: StandaloneSchedulerBackend =>
+          backend.client.workerLastHeartbeat(sc.applicationId, buf) match {
+            case Some(workerLastHeartbeats) =>
+              for ((executorId, workerLastHeartbeat) <- buf zip workerLastHeartbeats) {
+                if (now - workerLastHeartbeat > executorTimeoutMs / 2) {
+                  val lastSeenMs = executorLastSeen.get(executorId).get
+                  killExecutor(executorId, now - lastSeenMs)
+                  waitingList.remove(executorId)
+                } else {
+                  waitingList(executorId) = workerLastHeartbeat
+                }
+                executorLastSeen.remove(executorId)
+              }
+            case None =>
+              for (executorId <- buf) {
+                val lastSeenMs = executorLastSeen.get(executorId).get
+                killExecutor(executorId, now - lastSeenMs)
+                executorLastSeen.remove(executorId)
+                waitingList.remove(executorId)
+              }

Review Comment:
   Update the `if` condition [1] to avoid other scheduler backends entering the `else` condition.
    
   [1] https://github.com/apache/spark/pull/37411/files#diff-d1e46909e7b48d24379f8c373bc20e15f518da2f47655533d95bc947808fdd8fR299



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r940959881


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -77,17 +77,44 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
 
   private[spark] var scheduler: TaskScheduler = null
 
-  // executor ID -> timestamp of when the last heartbeat from this executor was received
+  /**
+   * [SPARK-39984]
+   * Please make sure the intersection between `executorLastSeen` and `waitingList` is an empty set.
+   * If the intersection is not empty, it is possible to never kill the executor until the executor
+   * recovers. When an executor is in both `executorLastSeen` and `waitingList`, the value of
+   * `workerLastHeartbeat` in waitingList may update if the worker sends heartbeats to master
+   * normally.
+   *
+   * `executorLastSeen`:
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from this executor was received
+   *
+   *  `waitingList`: executor ID -> WorkerLastHeartbeat
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from the worker was received
+   *
+   * when driver does not receive any heartbeat from an executor for `executorTimeoutMs` seconds,
+   * the driver will ask master for the last heartbeat from the worker which the executor is running
+   * on.
+   */
   private val executorLastSeen = new HashMap[String, Long]
+  private val waitingList = new HashMap[String, Long]
 
   private val executorTimeoutMs = sc.conf.get(
     config.STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT
-  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s"))
+  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_EXECUTOR_TIMEOUT)}s"))
 
   private val checkTimeoutIntervalMs = sc.conf.get(Network.NETWORK_TIMEOUT_INTERVAL)
 
   private val executorHeartbeatIntervalMs = sc.conf.get(config.EXECUTOR_HEARTBEAT_INTERVAL)
 
+  /**
+   * Currently, [SPARK-39984] is only for StandaloneSchedulerBackend.
+   */
+  private val checkWorkerLastHeartbeat =
+    sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT) &&
+      sc.schedulerBackend.isInstanceOf[StandaloneSchedulerBackend]

Review Comment:
   Thank you for pointing this out! I have fixed this in https://github.com/apache/spark/pull/37411/commits/9afdcfa5b71d2970ed7518f770aedadd3ccb82f1.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r940733545


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +222,131 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms",
+                causedByApp = !sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT))))
+
+          // LocalSchedulerBackend is used locally and only has one single executor
+          case _: LocalSchedulerBackend =>
+
+          case other => throw new UnsupportedOperationException(
+            s"Unknown scheduler backend: ${other.getClass}")
+        }
+      }
+    })
+  }
+
   private def expireDeadHosts(): Unit = {
+  /**
+   * [SC-105641]
+   * Originally, the driver’s HeartbeatReceiver will expire an executor if it does not receive any
+   * heartbeat from the executor for 120 seconds. However, 120 seconds is too long, but we will face
+   * other challenges when we try to lower the timeout threshold. To elaborate, when an executor is
+   * performing full GC, it cannot send/reply any message. Next paragraphs describe the solution to
+   * detect network disconnection between driver and executor in a short time.
+   *
+   * An executor is running on a worker but in different JVMs, and a driver is running on a master
+   * but in different JVMs. Hence, the network connection between driver/executor and master/worker
+   * is the same. Because executor and worker are running on different JVMs, worker can still send
+   * heartbeat to master when executor performs GC.
+   *
+   * For new Heartbeat Receiver, if driver does not receive any heartbeat from the executor for
+   * `executorTimeoutMs` (default: 60s) seconds, HeartbeatReceiver will send a request to master to
+   * ask for the latest heartbeat from the worker which the executor runs on `workerLastHeartbeat`.
+   * HeartbeatReceiver can determine whether the heartbeat loss is caused by network issues or other
+   * issues (e.g. GC). If the heartbeat loss is not caused by network issues, the HeartbeatReceiver
+   * will put the executor into a waitingList rather than expiring it immediately.
+   *
+   * [Note]: Definition of `network issues`
+   * Here, the definition `network issues` is the issues that related to network directly. If the
+   * network is connected, the issues do not included in `network issues`. For example, an
+   * executor's JVM is closed by a problematic task, so the JVM will notify driver that the socket
+   * is closed. If the network is connected, driver will receive the notification and trigger the
+   * function `onDisconnected`. This issue is not a `network issue` because the network is
+   * connected.
+   *
+   * [Warning 1]
+   * Worker will send heartbeats to Master every (conf.get(WORKER_TIMEOUT) * 1000 / 4) milliseconds.
+   * Check deploy/worker/Worker.scala for more details. This new mechanism design is based on the
+   * assumption: (executorTimeoutMs / 2) > (conf.get(WORKER_TIMEOUT) * 1000 / 4).
+   *
+   * [Warning 2]
+   * Not every deployment method schedules driver on master.
+   */
     logTrace("Checking for hosts with no recent heartbeats in HeartbeatReceiver.")
     val now = clock.getTimeMillis()
-    for ((executorId, lastSeenMs) <- executorLastSeen) {
-      if (now - lastSeenMs > executorTimeoutMs) {
-        logWarning(s"Removing executor $executorId with no recent heartbeats: " +
-          s"${now - lastSeenMs} ms exceeds timeout $executorTimeoutMs ms")
-        // Asynchronously kill the executor to avoid blocking the current thread
-        killExecutorThread.submit(new Runnable {
-          override def run(): Unit = Utils.tryLogNonFatalError {
-            // Note: we want to get an executor back after expiring this one,
-            // so do not simply call `sc.killExecutor` here (SPARK-8119)
-            sc.killAndReplaceExecutor(executorId)
-            // SPARK-27348: in case of the executors which are not gracefully shut down,
-            // we should remove lost executors from CoarseGrainedSchedulerBackend manually
-            // here to guarantee two things:
-            // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
-            //    a lost executor instead of waiting for disconnect message
-            // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
-            //    those executors to avoid app hang
-            sc.schedulerBackend match {
-              case backend: CoarseGrainedSchedulerBackend =>
-                backend.driverEndpoint.send(RemoveExecutor(executorId,
-                  ExecutorProcessLost(
-                    s"Executor heartbeat timed out after ${now - lastSeenMs} ms")))
-
-              // LocalSchedulerBackend is used locally and only has one single executor
-              case _: LocalSchedulerBackend =>
-
-              case other => throw new UnsupportedOperationException(
-                s"Unknown scheduler backend: ${other.getClass}")
-            }
+    if (!sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT)) {

Review Comment:
   Resolved.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r940758645


##########
core/src/main/scala/org/apache/spark/internal/config/package.scala:
##########
@@ -2398,4 +2398,11 @@ package object config {
       .version("3.3.0")
       .intConf
       .createWithDefault(5)
+
+  private[spark] val HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT =
+    ConfigBuilder("spark.driver.heartbeat.checkWorkerLastHeartbeat")
+      .internal()
+      .version("3.4.0")
+      .booleanConf
+      .createWithDefault(true)

Review Comment:
   I will update the default value to false when this PR is ready and passes CI/CD 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r953803448


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -77,17 +77,61 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
 
   private[spark] var scheduler: TaskScheduler = null
 
-  // executor ID -> timestamp of when the last heartbeat from this executor was received
+  /**
+   * [SPARK-39984]
+   * Please make sure the intersection between `executorLastSeen` and `executorExpiryCandidates` is
+   * an empty set. If the intersection is not empty, it is possible to never kill the executor until
+   * the executor recovers. When an executor is in both `executorLastSeen` and
+   * `executorExpiryCandidates`, the value of `workerLastHeartbeat` in `executorExpiryCandidates`
+   * may update if the worker sends heartbeats to master normally.
+   *
+   * `executorLastSeen`:
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from this executor was received
+   *
+   * `executorExpiryCandidates`: executor ID -> WorkerLastHeartbeat
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from the worker was received
+   *
+   * when driver does not receive any heartbeat from an executor for `executorTimeoutMs` seconds,
+   * the driver will ask master for the last heartbeat from the worker which the executor is running
+   * on.
+   */
   private val executorLastSeen = new HashMap[String, Long]
+  private val executorExpiryCandidates = new HashMap[String, Long]
 
   private val executorTimeoutMs = sc.conf.get(
     config.STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT
-  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s"))
+  ).getOrElse(
+    sc.conf.get(Network.NETWORK_EXECUTOR_TIMEOUT) match {
+      case Some(executorTimeout) => executorTimeout
+      case None => Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s")
+    }
+  )
 
   private val checkTimeoutIntervalMs = sc.conf.get(Network.NETWORK_TIMEOUT_INTERVAL)
 
   private val executorHeartbeatIntervalMs = sc.conf.get(config.EXECUTOR_HEARTBEAT_INTERVAL)
 
+  /**
+   * Currently, [SPARK-39984] is only for StandaloneSchedulerBackend.
+   *
+   * `checkWorkerLastHeartbeat`: A flag to enable two-phase executor timeout.
+   * `expiryCandidatesTimeout`: The timeout used for executorExpiryCandidates.
+   */
+  private val checkWorkerLastHeartbeat = {
+    val isEnabled = sc.conf.get(config.HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT)
+    if (isEnabled) logWarning(s"Keep `expiryCandidatesTimeout` larger than `HEARTBEAT_MILLIS` in" +
+      s"deploy/worker/Worker.scala to know whether master lost any heartbeat from the" +
+      s"worker or not.")
+    isEnabled
+  }
+
+  private val expiryCandidatesTimeout = checkWorkerLastHeartbeat match {
+    case true => sc.conf.get(config.HEARTBEAT_EXPIRY_CANDIDATES_TIMEOUT)

Review Comment:
   Shall we move this warning to here? And reword it like this:
   
   "Worker heartbeat check enabled. Note it only works normally if the configured ${config.HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT.key} is larger than worker's heartbeat interval."
   



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] mridulm commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r942900759


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -77,17 +77,44 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
 
   private[spark] var scheduler: TaskScheduler = null
 
-  // executor ID -> timestamp of when the last heartbeat from this executor was received
+  /**
+   * [SPARK-39984]
+   * Please make sure the intersection between `executorLastSeen` and `waitingList` is an empty set.
+   * If the intersection is not empty, it is possible to never kill the executor until the executor
+   * recovers. When an executor is in both `executorLastSeen` and `waitingList`, the value of
+   * `workerLastHeartbeat` in waitingList may update if the worker sends heartbeats to master
+   * normally.
+   *
+   * `executorLastSeen`:
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from this executor was received
+   *
+   *  `waitingList`: executor ID -> WorkerLastHeartbeat
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from the worker was received
+   *
+   * when driver does not receive any heartbeat from an executor for `executorTimeoutMs` seconds,
+   * the driver will ask master for the last heartbeat from the worker which the executor is running
+   * on.
+   */
   private val executorLastSeen = new HashMap[String, Long]
+  private val waitingList = new HashMap[String, Long]
 
   private val executorTimeoutMs = sc.conf.get(
     config.STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT
-  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s"))
+  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_EXECUTOR_TIMEOUT)}s"))
 
   private val checkTimeoutIntervalMs = sc.conf.get(Network.NETWORK_TIMEOUT_INTERVAL)
 
   private val executorHeartbeatIntervalMs = sc.conf.get(config.EXECUTOR_HEARTBEAT_INTERVAL)
 
+  /**
+   * Currently, [SPARK-39984] is only for StandaloneSchedulerBackend.
+   */
+  private val checkWorkerLastHeartbeat =
+    sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT) &&
+      sc.schedulerBackend.isInstanceOf[StandaloneSchedulerBackend]

Review Comment:
   Note that you can actually have `ExecutorRegistered` before `sc.schedulerBackend` is set - this is essentially a race condition. This is why we have a `if (scheduler != null) {` check while handling `Heartbeat` (which comes after registration).



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] mridulm commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r943285183


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +242,137 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            val isStandalone = backend.isInstanceOf[StandaloneSchedulerBackend]
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms",
+                causedByApp = !checkWorkerLastHeartbeat || !isStandalone)))
+
+          // LocalSchedulerBackend is used locally and only has one single executor
+          case _: LocalSchedulerBackend =>
+
+          case other => throw new UnsupportedOperationException(
+            s"Unknown scheduler backend: ${other.getClass}")
+        }
+      }
+    })
+  }
+
+  private def removeExecutorFromWaitingList(executorId: String): Unit = {
+    val isStandalone = sc.schedulerBackend.isInstanceOf[StandaloneSchedulerBackend]
+    if (checkWorkerLastHeartbeat && isStandalone) {
+      waitingList.remove(executorId)
+    }
+  }
+
   private def expireDeadHosts(): Unit = {
+  /**
+   * [SPARK-39984]
+   * Originally, the driver’s HeartbeatReceiver will expire an executor if it does not receive any
+   * heartbeat from the executor for 120 seconds. However, 120 seconds is too long, but we will face
+   * other challenges when we try to lower the timeout threshold. To elaborate, when an executor is
+   * performing full GC, it cannot send/reply any message. Next paragraphs describe the solution to
+   * detect network disconnection between driver and executor in a short time.
+   *
+   * An executor is running on a worker but in different JVMs, and a driver is running on a master
+   * but in different JVMs. Hence, the network connection between driver/executor and master/worker
+   * is the same. Because executor and worker are running on different JVMs, worker can still send
+   * heartbeat to master when executor performs GC.
+   *
+   * For new Heartbeat Receiver, if driver does not receive any heartbeat from the executor for
+   * `executorTimeoutMs` (default: 60s) seconds, HeartbeatReceiver will send a request to master to
+   * ask for the latest heartbeat from the worker which the executor runs on `workerLastHeartbeat`.
+   * HeartbeatReceiver can determine whether the heartbeat loss is caused by network issues or other
+   * issues (e.g. GC). If the heartbeat loss is not caused by network issues, the HeartbeatReceiver
+   * will put the executor into a waitingList rather than expiring it immediately.
+   *
+   * [Note]: Definition of `network issues`
+   * Here, the definition `network issues` is the issues that related to network directly. If the
+   * network is connected, the issues do not included in `network issues`. For example, an
+   * executor's JVM is closed by a problematic task, so the JVM will notify driver that the socket
+   * is closed. If the network is connected, driver will receive the notification and trigger the
+   * function `onDisconnected`. This issue is not a `network issue` because the network is
+   * connected.
+   *
+   * [Warning 1]
+   * Worker will send heartbeats to Master every (conf.get(WORKER_TIMEOUT) * 1000 / 4) milliseconds.
+   * Check deploy/worker/Worker.scala for more details. This new mechanism design is based on the
+   * assumption: `waitingListTimeout` > (conf.get(WORKER_TIMEOUT) * 1000 / 4).
+   *
+   * [Warning 2]
+   * Not every deployment method schedules driver on master.
+   */

Review Comment:
   Yes, log a warning



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r960646158


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -77,17 +77,61 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
 
   private[spark] var scheduler: TaskScheduler = null
 
-  // executor ID -> timestamp of when the last heartbeat from this executor was received
+  /**
+   * [SPARK-39984]
+   * Please make sure the intersection between `executorLastSeen` and `executorExpiryCandidates` is
+   * an empty set. If the intersection is not empty, it is possible to never kill the executor until
+   * the executor recovers. When an executor is in both `executorLastSeen` and
+   * `executorExpiryCandidates`, the value of `workerLastHeartbeat` in `executorExpiryCandidates`
+   * may update if the worker sends heartbeats to master normally.
+   *
+   * `executorLastSeen`:
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from this executor was received
+   *
+   * `executorExpiryCandidates`: executor ID -> WorkerLastHeartbeat
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from the worker was received
+   *
+   * when driver does not receive any heartbeat from an executor for `executorTimeoutMs` seconds,
+   * the driver will ask master for the last heartbeat from the worker which the executor is running
+   * on.
+   */
   private val executorLastSeen = new HashMap[String, Long]
+  private val executorExpiryCandidates = new HashMap[String, Long]
 
   private val executorTimeoutMs = sc.conf.get(
     config.STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT
-  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s"))
+  ).getOrElse(
+    sc.conf.get(Network.NETWORK_EXECUTOR_TIMEOUT) match {
+      case Some(executorTimeout) => executorTimeout
+      case None => Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s")
+    }
+  )
 
   private val checkTimeoutIntervalMs = sc.conf.get(Network.NETWORK_TIMEOUT_INTERVAL)
 
   private val executorHeartbeatIntervalMs = sc.conf.get(config.EXECUTOR_HEARTBEAT_INTERVAL)
 
+  /**
+   * Currently, [SPARK-39984] is only for StandaloneSchedulerBackend.
+   *
+   * `checkWorkerLastHeartbeat`: A flag to enable two-phase executor timeout.
+   * `expiryCandidatesTimeout`: The timeout used for executorExpiryCandidates.
+   */
+  private val checkWorkerLastHeartbeat = {

Review Comment:
   > However, if the first expireDeadHosts trigger is prior to scheduler backend initialization, the value of checkWorkerLastHeartbeat will be false
   
   I thought there would be an initial delay for the first `expireDeadHosts` so `checkWorkerLastHeartbeat` is less likely to return false. But seems it doesn't have the initial delay, though I think it doesn't make sense.
   
   I actually think we can set the initial delay to its check interval: `checkTimeoutIntervalMs`.
   



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r960664710


##########
core/src/main/scala/org/apache/spark/internal/config/package.scala:
##########
@@ -2422,4 +2422,24 @@ package object config {
       .version("3.4.0")
       .timeConf(TimeUnit.MILLISECONDS)
       .createWithDefaultString("5s")
+
+  private[spark] val HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT =
+    ConfigBuilder("spark.driver.heartbeat.checkWorkerLastHeartbeat")
+      .doc("If this config is set to true, heartbeat receiver will check worker's heartbeat when" +
+        "the receiver does not receive any heartbeat from an executor during a period. In" +
+        "addition, this config is only for standalone scheduler. See [SPARK-39984] for more" +
+        "details.")
+      .internal()
+      .version("3.4.0")
+      .booleanConf
+      .createWithDefault(false)
+
+  private[spark] val HEARTBEAT_EXPIRY_CANDIDATES_TIMEOUT =
+    ConfigBuilder("spark.driver.heartbeat.expiryCandidatesTimeout")
+      .doc("This config is a timeout used for heartbeat receiver `executorExpiryCandidates`. Be" +
+        "effective only when HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT is enabled. See" +

Review Comment:
   ```suggestion
           s"effective only when ${HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT.key} is enabled. See" +
   ```



##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -77,17 +77,61 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
 
   private[spark] var scheduler: TaskScheduler = null
 
-  // executor ID -> timestamp of when the last heartbeat from this executor was received
+  /**
+   * [SPARK-39984]
+   * Please make sure the intersection between `executorLastSeen` and `executorExpiryCandidates` is
+   * an empty set. If the intersection is not empty, it is possible to never kill the executor until
+   * the executor recovers. When an executor is in both `executorLastSeen` and
+   * `executorExpiryCandidates`, the value of `workerLastHeartbeat` in `executorExpiryCandidates`
+   * may update if the worker sends heartbeats to master normally.
+   *
+   * `executorLastSeen`:
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from this executor was received
+   *
+   * `executorExpiryCandidates`: executor ID -> WorkerLastHeartbeat
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from the worker was received
+   *
+   * when driver does not receive any heartbeat from an executor for `executorTimeoutMs` seconds,
+   * the driver will ask master for the last heartbeat from the worker which the executor is running
+   * on.
+   */
   private val executorLastSeen = new HashMap[String, Long]
+  private val executorExpiryCandidates = new HashMap[String, Long]
 
   private val executorTimeoutMs = sc.conf.get(
     config.STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT
-  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s"))
+  ).getOrElse(
+    sc.conf.get(Network.NETWORK_EXECUTOR_TIMEOUT) match {
+      case Some(executorTimeout) => executorTimeout
+      case None => Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s")
+    }
+  )
 
   private val checkTimeoutIntervalMs = sc.conf.get(Network.NETWORK_TIMEOUT_INTERVAL)
 
   private val executorHeartbeatIntervalMs = sc.conf.get(config.EXECUTOR_HEARTBEAT_INTERVAL)
 
+  /**
+   * Currently, [SPARK-39984] is only for StandaloneSchedulerBackend.
+   *
+   * `checkWorkerLastHeartbeat`: A flag to enable two-phase executor timeout.
+   * `expiryCandidatesTimeout`: The timeout used for executorExpiryCandidates.
+   */
+  private val checkWorkerLastHeartbeat = {

Review Comment:
   > However, it cannot totally prevent this edge case
   
   Theoretically, I agree it doesn't but given that checkTimeoutIntervalMs by default is 60s it's almost impossible to have scheduler backend not initialized when the first expireDeadHosts raised.
   
   
   > Hence, my thought is to define
   checkWorkerLastHeartbeat as a function, and we can totally prevent the edge case. Does it make sense?
   
   I don't think it makes a big difference with the already defined `isStandalone()` function. Maybe just leave it as is if we have no better idea.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r940962990


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -77,17 +77,44 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
 
   private[spark] var scheduler: TaskScheduler = null
 
-  // executor ID -> timestamp of when the last heartbeat from this executor was received
+  /**
+   * [SPARK-39984]
+   * Please make sure the intersection between `executorLastSeen` and `waitingList` is an empty set.
+   * If the intersection is not empty, it is possible to never kill the executor until the executor
+   * recovers. When an executor is in both `executorLastSeen` and `waitingList`, the value of
+   * `workerLastHeartbeat` in waitingList may update if the worker sends heartbeats to master
+   * normally.
+   *
+   * `executorLastSeen`:
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from this executor was received
+   *
+   *  `waitingList`: executor ID -> WorkerLastHeartbeat
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from the worker was received
+   *
+   * when driver does not receive any heartbeat from an executor for `executorTimeoutMs` seconds,
+   * the driver will ask master for the last heartbeat from the worker which the executor is running
+   * on.
+   */
   private val executorLastSeen = new HashMap[String, Long]
+  private val waitingList = new HashMap[String, Long]
 
   private val executorTimeoutMs = sc.conf.get(
     config.STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT
-  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s"))
+  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_EXECUTOR_TIMEOUT)}s"))
 
   private val checkTimeoutIntervalMs = sc.conf.get(Network.NETWORK_TIMEOUT_INTERVAL)
 
   private val executorHeartbeatIntervalMs = sc.conf.get(config.EXECUTOR_HEARTBEAT_INTERVAL)
 
+  /**
+   * Currently, [SPARK-39984] is only for StandaloneSchedulerBackend.
+   */
+  private val checkWorkerLastHeartbeat =
+    sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT) &&
+      sc.schedulerBackend.isInstanceOf[StandaloneSchedulerBackend]

Review Comment:
   TODO: I will check whether SchedulerBackend is initialized or not before HeartbeatReceiver receives `ExecutorRegistered` / `ExecutorRemoved` / `Heartbeat`.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] mridulm commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r940866631


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +222,131 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms",
+                causedByApp = !sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT))))
+
+          // LocalSchedulerBackend is used locally and only has one single executor
+          case _: LocalSchedulerBackend =>
+
+          case other => throw new UnsupportedOperationException(
+            s"Unknown scheduler backend: ${other.getClass}")
+        }
+      }
+    })
+  }
+
   private def expireDeadHosts(): Unit = {
+  /**
+   * [SC-105641]
+   * Originally, the driver’s HeartbeatReceiver will expire an executor if it does not receive any
+   * heartbeat from the executor for 120 seconds. However, 120 seconds is too long, but we will face
+   * other challenges when we try to lower the timeout threshold. To elaborate, when an executor is
+   * performing full GC, it cannot send/reply any message. Next paragraphs describe the solution to
+   * detect network disconnection between driver and executor in a short time.
+   *
+   * An executor is running on a worker but in different JVMs, and a driver is running on a master
+   * but in different JVMs. Hence, the network connection between driver/executor and master/worker
+   * is the same. Because executor and worker are running on different JVMs, worker can still send
+   * heartbeat to master when executor performs GC.
+   *
+   * For new Heartbeat Receiver, if driver does not receive any heartbeat from the executor for
+   * `executorTimeoutMs` (default: 60s) seconds, HeartbeatReceiver will send a request to master to
+   * ask for the latest heartbeat from the worker which the executor runs on `workerLastHeartbeat`.
+   * HeartbeatReceiver can determine whether the heartbeat loss is caused by network issues or other
+   * issues (e.g. GC). If the heartbeat loss is not caused by network issues, the HeartbeatReceiver
+   * will put the executor into a waitingList rather than expiring it immediately.
+   *
+   * [Note]: Definition of `network issues`
+   * Here, the definition `network issues` is the issues that related to network directly. If the
+   * network is connected, the issues do not included in `network issues`. For example, an
+   * executor's JVM is closed by a problematic task, so the JVM will notify driver that the socket
+   * is closed. If the network is connected, driver will receive the notification and trigger the
+   * function `onDisconnected`. This issue is not a `network issue` because the network is
+   * connected.
+   *
+   * [Warning 1]
+   * Worker will send heartbeats to Master every (conf.get(WORKER_TIMEOUT) * 1000 / 4) milliseconds.
+   * Check deploy/worker/Worker.scala for more details. This new mechanism design is based on the
+   * assumption: (executorTimeoutMs / 2) > (conf.get(WORKER_TIMEOUT) * 1000 / 4).
+   *
+   * [Warning 2]
+   * Not every deployment method schedules driver on master.
+   */
     logTrace("Checking for hosts with no recent heartbeats in HeartbeatReceiver.")
     val now = clock.getTimeMillis()
-    for ((executorId, lastSeenMs) <- executorLastSeen) {
-      if (now - lastSeenMs > executorTimeoutMs) {
-        logWarning(s"Removing executor $executorId with no recent heartbeats: " +
-          s"${now - lastSeenMs} ms exceeds timeout $executorTimeoutMs ms")
-        // Asynchronously kill the executor to avoid blocking the current thread
-        killExecutorThread.submit(new Runnable {
-          override def run(): Unit = Utils.tryLogNonFatalError {
-            // Note: we want to get an executor back after expiring this one,
-            // so do not simply call `sc.killExecutor` here (SPARK-8119)
-            sc.killAndReplaceExecutor(executorId)
-            // SPARK-27348: in case of the executors which are not gracefully shut down,
-            // we should remove lost executors from CoarseGrainedSchedulerBackend manually
-            // here to guarantee two things:
-            // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
-            //    a lost executor instead of waiting for disconnect message
-            // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
-            //    those executors to avoid app hang
-            sc.schedulerBackend match {
-              case backend: CoarseGrainedSchedulerBackend =>
-                backend.driverEndpoint.send(RemoveExecutor(executorId,
-                  ExecutorProcessLost(
-                    s"Executor heartbeat timed out after ${now - lastSeenMs} ms")))
-
-              // LocalSchedulerBackend is used locally and only has one single executor
-              case _: LocalSchedulerBackend =>
-
-              case other => throw new UnsupportedOperationException(
-                s"Unknown scheduler backend: ${other.getClass}")
-            }
+    if (!sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT)) {
+      for ((executorId, lastSeenMs) <- executorLastSeen) {
+        if (now - lastSeenMs > executorTimeoutMs) {
+          killExecutor(executorId, now - lastSeenMs)
+          waitingList.remove(executorId)
+          executorLastSeen.remove(executorId)
+        }
+      }
+    } else {
+      for ((executorId, workerLastHeartbeat) <- waitingList) {
+        if (now - workerLastHeartbeat > executorTimeoutMs / 2) {
+          killExecutor(executorId, now - workerLastHeartbeat)
+          waitingList.remove(executorId)
+          executorLastSeen.remove(executorId)
+        }
+      }
+
+      val buf = new ArrayBuffer[String]()
+      for ((executorId, lastSeenMs) <- executorLastSeen) {
+        if (now - lastSeenMs > executorTimeoutMs) {
+          sc.schedulerBackend match {
+            case _: StandaloneSchedulerBackend =>
+              buf += executorId
+            case _ =>
+              killExecutor(executorId, now - lastSeenMs)
+              waitingList.remove(executorId)
+              executorLastSeen.remove(executorId)
+          }
+        }
+      }
+
+      sc.schedulerBackend match {
+        case backend: StandaloneSchedulerBackend =>
+          backend.client.workerLastHeartbeat(sc.applicationId, buf) match {
+            case Some(workerLastHeartbeats) =>
+              for ((executorId, workerLastHeartbeat) <- buf zip workerLastHeartbeats) {
+                if (now - workerLastHeartbeat > executorTimeoutMs / 2) {
+                  val lastSeenMs = executorLastSeen.get(executorId).get
+                  killExecutor(executorId, now - lastSeenMs)
+                  waitingList.remove(executorId)
+                } else {
+                  waitingList(executorId) = workerLastHeartbeat
+                }
+                executorLastSeen.remove(executorId)
+              }
+            case None =>
+              for (executorId <- buf) {
+                val lastSeenMs = executorLastSeen.get(executorId).get
+                killExecutor(executorId, now - lastSeenMs)
+                executorLastSeen.remove(executorId)
+                waitingList.remove(executorId)
+              }

Review Comment:
   This (and other cases in this class) would get impacted due to lack of scheduler backend when the heatbeat receiver is created - please do revisit them after fixing that issue.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on PR #37411:
URL: https://github.com/apache/spark/pull/37411#issuecomment-1213650645

   Hi @mridulm, here are the JIRA tickets. Thank you!
   
   YARN: https://issues.apache.org/jira/browse/SPARK-40068
   k8s: https://issues.apache.org/jira/browse/SPARK-40069


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r940757891


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +222,131 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms",
+                causedByApp = !sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT))))
+
+          // LocalSchedulerBackend is used locally and only has one single executor
+          case _: LocalSchedulerBackend =>
+
+          case other => throw new UnsupportedOperationException(
+            s"Unknown scheduler backend: ${other.getClass}")
+        }
+      }
+    })
+  }
+
   private def expireDeadHosts(): Unit = {
+  /**
+   * [SC-105641]
+   * Originally, the driver’s HeartbeatReceiver will expire an executor if it does not receive any
+   * heartbeat from the executor for 120 seconds. However, 120 seconds is too long, but we will face
+   * other challenges when we try to lower the timeout threshold. To elaborate, when an executor is
+   * performing full GC, it cannot send/reply any message. Next paragraphs describe the solution to
+   * detect network disconnection between driver and executor in a short time.
+   *
+   * An executor is running on a worker but in different JVMs, and a driver is running on a master
+   * but in different JVMs. Hence, the network connection between driver/executor and master/worker
+   * is the same. Because executor and worker are running on different JVMs, worker can still send
+   * heartbeat to master when executor performs GC.
+   *
+   * For new Heartbeat Receiver, if driver does not receive any heartbeat from the executor for
+   * `executorTimeoutMs` (default: 60s) seconds, HeartbeatReceiver will send a request to master to
+   * ask for the latest heartbeat from the worker which the executor runs on `workerLastHeartbeat`.
+   * HeartbeatReceiver can determine whether the heartbeat loss is caused by network issues or other
+   * issues (e.g. GC). If the heartbeat loss is not caused by network issues, the HeartbeatReceiver
+   * will put the executor into a waitingList rather than expiring it immediately.
+   *
+   * [Note]: Definition of `network issues`
+   * Here, the definition `network issues` is the issues that related to network directly. If the
+   * network is connected, the issues do not included in `network issues`. For example, an
+   * executor's JVM is closed by a problematic task, so the JVM will notify driver that the socket
+   * is closed. If the network is connected, driver will receive the notification and trigger the
+   * function `onDisconnected`. This issue is not a `network issue` because the network is
+   * connected.
+   *
+   * [Warning 1]
+   * Worker will send heartbeats to Master every (conf.get(WORKER_TIMEOUT) * 1000 / 4) milliseconds.
+   * Check deploy/worker/Worker.scala for more details. This new mechanism design is based on the
+   * assumption: (executorTimeoutMs / 2) > (conf.get(WORKER_TIMEOUT) * 1000 / 4).
+   *
+   * [Warning 2]
+   * Not every deployment method schedules driver on master.
+   */
     logTrace("Checking for hosts with no recent heartbeats in HeartbeatReceiver.")
     val now = clock.getTimeMillis()
-    for ((executorId, lastSeenMs) <- executorLastSeen) {
-      if (now - lastSeenMs > executorTimeoutMs) {
-        logWarning(s"Removing executor $executorId with no recent heartbeats: " +
-          s"${now - lastSeenMs} ms exceeds timeout $executorTimeoutMs ms")
-        // Asynchronously kill the executor to avoid blocking the current thread
-        killExecutorThread.submit(new Runnable {
-          override def run(): Unit = Utils.tryLogNonFatalError {
-            // Note: we want to get an executor back after expiring this one,
-            // so do not simply call `sc.killExecutor` here (SPARK-8119)
-            sc.killAndReplaceExecutor(executorId)
-            // SPARK-27348: in case of the executors which are not gracefully shut down,
-            // we should remove lost executors from CoarseGrainedSchedulerBackend manually
-            // here to guarantee two things:
-            // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
-            //    a lost executor instead of waiting for disconnect message
-            // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
-            //    those executors to avoid app hang
-            sc.schedulerBackend match {
-              case backend: CoarseGrainedSchedulerBackend =>
-                backend.driverEndpoint.send(RemoveExecutor(executorId,
-                  ExecutorProcessLost(
-                    s"Executor heartbeat timed out after ${now - lastSeenMs} ms")))
-
-              // LocalSchedulerBackend is used locally and only has one single executor
-              case _: LocalSchedulerBackend =>
-
-              case other => throw new UnsupportedOperationException(
-                s"Unknown scheduler backend: ${other.getClass}")
-            }
+    if (!sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT)) {
+      for ((executorId, lastSeenMs) <- executorLastSeen) {
+        if (now - lastSeenMs > executorTimeoutMs) {
+          killExecutor(executorId, now - lastSeenMs)
+          waitingList.remove(executorId)
+          executorLastSeen.remove(executorId)
+        }
+      }
+    } else {
+      for ((executorId, workerLastHeartbeat) <- waitingList) {
+        if (now - workerLastHeartbeat > executorTimeoutMs / 2) {

Review Comment:
   If an executor lasts in `waitingList` for more than `executorTimeoutMs / 2` seconds, it will be killed. I just want to choose a value less than `executorTimeoutMs`, and thus I choose `executorTimeoutMs / 2`. Whether we need an additional variable to set the value of timeout in `waitingList`?



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] mridulm commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r943290018


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +242,137 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            val isStandalone = backend.isInstanceOf[StandaloneSchedulerBackend]
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms",
+                causedByApp = !checkWorkerLastHeartbeat || !isStandalone)))

Review Comment:
   
   For case (1), the task will keep getting rerun on a different executors - where it will keep hanging repeatedly - if the hang is due to the user code.
   
   You are right, (2) cannot be detected - but speculative execution might help with that if it is related to the specific node (if it is related to user code/data, it will hang indefinitely until user intervention).
   
   For (3), we get an executor removed, and will get handled as existing (not related to this PR).
   
   



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r943745499


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +242,137 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            val isStandalone = backend.isInstanceOf[StandaloneSchedulerBackend]
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms",
+                causedByApp = !checkWorkerLastHeartbeat || !isStandalone)))

Review Comment:
   Updated `causedByApp` to true (its default value). Will update `causedByApp` when we have a hanging task detector in executor.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] mridulm commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r940862802


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -77,17 +77,44 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
 
   private[spark] var scheduler: TaskScheduler = null
 
-  // executor ID -> timestamp of when the last heartbeat from this executor was received
+  /**
+   * [SPARK-39984]
+   * Please make sure the intersection between `executorLastSeen` and `waitingList` is an empty set.
+   * If the intersection is not empty, it is possible to never kill the executor until the executor
+   * recovers. When an executor is in both `executorLastSeen` and `waitingList`, the value of
+   * `workerLastHeartbeat` in waitingList may update if the worker sends heartbeats to master
+   * normally.
+   *
+   * `executorLastSeen`:
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from this executor was received
+   *
+   *  `waitingList`: executor ID -> WorkerLastHeartbeat
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from the worker was received
+   *
+   * when driver does not receive any heartbeat from an executor for `executorTimeoutMs` seconds,
+   * the driver will ask master for the last heartbeat from the worker which the executor is running
+   * on.
+   */
   private val executorLastSeen = new HashMap[String, Long]
+  private val waitingList = new HashMap[String, Long]
 
   private val executorTimeoutMs = sc.conf.get(
     config.STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT
-  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s"))
+  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_EXECUTOR_TIMEOUT)}s"))
 
   private val checkTimeoutIntervalMs = sc.conf.get(Network.NETWORK_TIMEOUT_INTERVAL)
 
   private val executorHeartbeatIntervalMs = sc.conf.get(config.EXECUTOR_HEARTBEAT_INTERVAL)
 
+  /**
+   * Currently, [SPARK-39984] is only for StandaloneSchedulerBackend.
+   */
+  private val checkWorkerLastHeartbeat =
+    sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT) &&
+      sc.schedulerBackend.isInstanceOf[StandaloneSchedulerBackend]

Review Comment:
   When `HeartbeatReceiver` is created in `SparkContext`, `_schedulerBackend` is not yet created/initialized - so we cannot depend on this here (`sc.schedulerBackend` will always return `null`)



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] mridulm commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r945364383


##########
core/src/main/scala/org/apache/spark/internal/config/Network.scala:
##########
@@ -51,6 +51,21 @@ private[spark] object Network {
       .timeConf(TimeUnit.MILLISECONDS)
       .createWithDefaultString(STORAGE_BLOCKMANAGER_TIMEOUTINTERVAL.defaultValueString)
 
+  private[spark] val NETWORK_EXECUTOR_TIMEOUT =
+    ConfigBuilder("spark.network.executorTimeout")
+      .version("3.4.0")
+      .timeConf(TimeUnit.MILLISECONDS)
+      .createOptional
+
+  private[spark] val HEARTBEAT_EXPIRY_CANDIDATES_TIMEOUT =
+    ConfigBuilder("spark.network.expiryCandidatesTimeout")
+      .doc("This config is a timeout used for heartbeat receiver `executorExpiryCandidates`. Be" +
+        "effective only when HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT is enabled. See" +
+        "[SPARK-39984] for more details")
+      .version("3.4.0")
+      .timeConf(TimeUnit.MILLISECONDS)
+      .createWithDefaultString("30s")

Review Comment:
   We should namespace these configs to heartbeat, thoughts @Ngone51  ?



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r946373236


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +222,131 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms",
+                causedByApp = !sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT))))
+
+          // LocalSchedulerBackend is used locally and only has one single executor
+          case _: LocalSchedulerBackend =>
+
+          case other => throw new UnsupportedOperationException(
+            s"Unknown scheduler backend: ${other.getClass}")
+        }
+      }
+    })
+  }
+
   private def expireDeadHosts(): Unit = {
+  /**
+   * [SC-105641]
+   * Originally, the driver’s HeartbeatReceiver will expire an executor if it does not receive any
+   * heartbeat from the executor for 120 seconds. However, 120 seconds is too long, but we will face
+   * other challenges when we try to lower the timeout threshold. To elaborate, when an executor is
+   * performing full GC, it cannot send/reply any message. Next paragraphs describe the solution to
+   * detect network disconnection between driver and executor in a short time.
+   *
+   * An executor is running on a worker but in different JVMs, and a driver is running on a master
+   * but in different JVMs. Hence, the network connection between driver/executor and master/worker
+   * is the same. Because executor and worker are running on different JVMs, worker can still send
+   * heartbeat to master when executor performs GC.
+   *
+   * For new Heartbeat Receiver, if driver does not receive any heartbeat from the executor for
+   * `executorTimeoutMs` (default: 60s) seconds, HeartbeatReceiver will send a request to master to
+   * ask for the latest heartbeat from the worker which the executor runs on `workerLastHeartbeat`.
+   * HeartbeatReceiver can determine whether the heartbeat loss is caused by network issues or other
+   * issues (e.g. GC). If the heartbeat loss is not caused by network issues, the HeartbeatReceiver
+   * will put the executor into a waitingList rather than expiring it immediately.
+   *
+   * [Note]: Definition of `network issues`
+   * Here, the definition `network issues` is the issues that related to network directly. If the
+   * network is connected, the issues do not included in `network issues`. For example, an
+   * executor's JVM is closed by a problematic task, so the JVM will notify driver that the socket
+   * is closed. If the network is connected, driver will receive the notification and trigger the
+   * function `onDisconnected`. This issue is not a `network issue` because the network is
+   * connected.
+   *
+   * [Warning 1]
+   * Worker will send heartbeats to Master every (conf.get(WORKER_TIMEOUT) * 1000 / 4) milliseconds.
+   * Check deploy/worker/Worker.scala for more details. This new mechanism design is based on the
+   * assumption: (executorTimeoutMs / 2) > (conf.get(WORKER_TIMEOUT) * 1000 / 4).
+   *
+   * [Warning 2]
+   * Not every deployment method schedules driver on master.
+   */
     logTrace("Checking for hosts with no recent heartbeats in HeartbeatReceiver.")
     val now = clock.getTimeMillis()
-    for ((executorId, lastSeenMs) <- executorLastSeen) {
-      if (now - lastSeenMs > executorTimeoutMs) {
-        logWarning(s"Removing executor $executorId with no recent heartbeats: " +
-          s"${now - lastSeenMs} ms exceeds timeout $executorTimeoutMs ms")
-        // Asynchronously kill the executor to avoid blocking the current thread
-        killExecutorThread.submit(new Runnable {
-          override def run(): Unit = Utils.tryLogNonFatalError {
-            // Note: we want to get an executor back after expiring this one,
-            // so do not simply call `sc.killExecutor` here (SPARK-8119)
-            sc.killAndReplaceExecutor(executorId)
-            // SPARK-27348: in case of the executors which are not gracefully shut down,
-            // we should remove lost executors from CoarseGrainedSchedulerBackend manually
-            // here to guarantee two things:
-            // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
-            //    a lost executor instead of waiting for disconnect message
-            // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
-            //    those executors to avoid app hang
-            sc.schedulerBackend match {
-              case backend: CoarseGrainedSchedulerBackend =>
-                backend.driverEndpoint.send(RemoveExecutor(executorId,
-                  ExecutorProcessLost(
-                    s"Executor heartbeat timed out after ${now - lastSeenMs} ms")))
-
-              // LocalSchedulerBackend is used locally and only has one single executor
-              case _: LocalSchedulerBackend =>
-
-              case other => throw new UnsupportedOperationException(
-                s"Unknown scheduler backend: ${other.getClass}")
-            }
+    if (!sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT)) {
+      for ((executorId, lastSeenMs) <- executorLastSeen) {
+        if (now - lastSeenMs > executorTimeoutMs) {
+          killExecutor(executorId, now - lastSeenMs)
+          waitingList.remove(executorId)
+          executorLastSeen.remove(executorId)
+        }
+      }
+    } else {
+      for ((executorId, workerLastHeartbeat) <- waitingList) {
+        if (now - workerLastHeartbeat > executorTimeoutMs / 2) {
+          killExecutor(executorId, now - workerLastHeartbeat)
+          waitingList.remove(executorId)
+          executorLastSeen.remove(executorId)
+        }
+      }
+
+      val buf = new ArrayBuffer[String]()
+      for ((executorId, lastSeenMs) <- executorLastSeen) {
+        if (now - lastSeenMs > executorTimeoutMs) {
+          sc.schedulerBackend match {
+            case _: StandaloneSchedulerBackend =>
+              buf += executorId
+            case _ =>
+              killExecutor(executorId, now - lastSeenMs)
+              waitingList.remove(executorId)
+              executorLastSeen.remove(executorId)
+          }
+        }
+      }
+
+      sc.schedulerBackend match {
+        case backend: StandaloneSchedulerBackend =>
+          backend.client.workerLastHeartbeat(sc.applicationId, buf) match {
+            case Some(workerLastHeartbeats) =>
+              for ((executorId, workerLastHeartbeat) <- buf zip workerLastHeartbeats) {
+                if (now - workerLastHeartbeat > executorTimeoutMs / 2) {
+                  val lastSeenMs = executorLastSeen.get(executorId).get
+                  killExecutor(executorId, now - lastSeenMs)
+                  waitingList.remove(executorId)
+                } else {
+                  waitingList(executorId) = workerLastHeartbeat
+                }
+                executorLastSeen.remove(executorId)
+              }
+            case None =>
+              for (executorId <- buf) {
+                val lastSeenMs = executorLastSeen.get(executorId).get
+                killExecutor(executorId, now - lastSeenMs)
+                executorLastSeen.remove(executorId)
+                waitingList.remove(executorId)
+              }

Review Comment:
   @Ngone51 Here is one of the discussions about the lack of scheduler backend.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] mridulm commented on pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
mridulm commented on PR #37411:
URL: https://github.com/apache/spark/pull/37411#issuecomment-1213647075

   Sounds good @kevin85421, wanted to make sure the approach is extensible to others.
   Can you please file follow up jira's for both ? Thx


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r953806281


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -77,17 +77,61 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
 
   private[spark] var scheduler: TaskScheduler = null
 
-  // executor ID -> timestamp of when the last heartbeat from this executor was received
+  /**
+   * [SPARK-39984]
+   * Please make sure the intersection between `executorLastSeen` and `executorExpiryCandidates` is
+   * an empty set. If the intersection is not empty, it is possible to never kill the executor until
+   * the executor recovers. When an executor is in both `executorLastSeen` and
+   * `executorExpiryCandidates`, the value of `workerLastHeartbeat` in `executorExpiryCandidates`
+   * may update if the worker sends heartbeats to master normally.
+   *
+   * `executorLastSeen`:
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from this executor was received
+   *
+   * `executorExpiryCandidates`: executor ID -> WorkerLastHeartbeat
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from the worker was received
+   *
+   * when driver does not receive any heartbeat from an executor for `executorTimeoutMs` seconds,
+   * the driver will ask master for the last heartbeat from the worker which the executor is running
+   * on.
+   */
   private val executorLastSeen = new HashMap[String, Long]
+  private val executorExpiryCandidates = new HashMap[String, Long]
 
   private val executorTimeoutMs = sc.conf.get(
     config.STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT
-  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s"))
+  ).getOrElse(
+    sc.conf.get(Network.NETWORK_EXECUTOR_TIMEOUT) match {
+      case Some(executorTimeout) => executorTimeout
+      case None => Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s")
+    }
+  )
 
   private val checkTimeoutIntervalMs = sc.conf.get(Network.NETWORK_TIMEOUT_INTERVAL)
 
   private val executorHeartbeatIntervalMs = sc.conf.get(config.EXECUTOR_HEARTBEAT_INTERVAL)
 
+  /**
+   * Currently, [SPARK-39984] is only for StandaloneSchedulerBackend.
+   *
+   * `checkWorkerLastHeartbeat`: A flag to enable two-phase executor timeout.
+   * `expiryCandidatesTimeout`: The timeout used for executorExpiryCandidates.
+   */
+  private val checkWorkerLastHeartbeat = {

Review Comment:
   Shall we make this a lazy val? Then, we can check `sc.schedulerBackend.isInstanceOf[StandaloneSchedulerBackend]` directly here instead of calling a fucntion. 



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r940785877


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -77,12 +77,32 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
 
   private[spark] var scheduler: TaskScheduler = null
 
-  // executor ID -> timestamp of when the last heartbeat from this executor was received
+  /**
+   * [SC-105641]
+   * Please make sure the intersection between `executorLastSeen` and `waitingList` is an empty set.
+   * If the intersection is not empty, it is possible to never kill the executor until the executor
+   * recovers. When an executor is in both `executorLastSeen` and `waitingList`, the value of
+   * `workerLastHeartbeat` in waitingList may update if the worker sends heartbeats to master
+   * normally.
+   *
+   * `executorLastSeen`:
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from this executor was received
+   *
+   *  `waitingList`: executor ID -> WorkerLastHeartbeat
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from the worker was received
+   *
+   * when driver does not receive any heartbeat from an executor for `executorTimeoutMs` seconds,
+   * the driver will ask master for the last heartbeat from the worker which the executor is running
+   * on.
+   */
   private val executorLastSeen = new HashMap[String, Long]
+  private val waitingList = new HashMap[String, Long]

Review Comment:
   Updated. 
   
   Originally, I updated/maintained `waitingList` no matter the value of `HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT` to make dynamic configuration implementation convenient. Currently, we only update/maintain `waitingList` when `checkWorkerLastHeartbeat` is true.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r942917348


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -77,17 +77,44 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
 
   private[spark] var scheduler: TaskScheduler = null
 
-  // executor ID -> timestamp of when the last heartbeat from this executor was received
+  /**
+   * [SPARK-39984]
+   * Please make sure the intersection between `executorLastSeen` and `waitingList` is an empty set.
+   * If the intersection is not empty, it is possible to never kill the executor until the executor
+   * recovers. When an executor is in both `executorLastSeen` and `waitingList`, the value of
+   * `workerLastHeartbeat` in waitingList may update if the worker sends heartbeats to master
+   * normally.
+   *
+   * `executorLastSeen`:
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from this executor was received
+   *
+   *  `waitingList`: executor ID -> WorkerLastHeartbeat
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from the worker was received
+   *
+   * when driver does not receive any heartbeat from an executor for `executorTimeoutMs` seconds,
+   * the driver will ask master for the last heartbeat from the worker which the executor is running
+   * on.
+   */
   private val executorLastSeen = new HashMap[String, Long]
+  private val waitingList = new HashMap[String, Long]
 
   private val executorTimeoutMs = sc.conf.get(
     config.STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT
-  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s"))
+  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_EXECUTOR_TIMEOUT)}s"))
 
   private val checkTimeoutIntervalMs = sc.conf.get(Network.NETWORK_TIMEOUT_INTERVAL)
 
   private val executorHeartbeatIntervalMs = sc.conf.get(config.EXECUTOR_HEARTBEAT_INTERVAL)
 
+  /**
+   * Currently, [SPARK-39984] is only for StandaloneSchedulerBackend.
+   */
+  private val checkWorkerLastHeartbeat =
+    sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT) &&
+      sc.schedulerBackend.isInstanceOf[StandaloneSchedulerBackend]

Review Comment:
   Hi @mridulm, the event `SparkListenerExecutorAdded` will only be posted by `LocalSchedulerBackend` ([Link](https://sourcegraph.com/github.com/apache/spark@b012cb722f6105a945e971bdc509f803a90b7419/-/blob/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala?L134)), `CoarseGrainedSchedulerBackend` ([Link](https://sourcegraph.com/github.com/apache/spark@b012cb722f6105a945e971bdc509f803a90b7419/-/blob/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala?L299:13)), and `MesosFineGrainedSchedulerBackend` ([Link](https://sourcegraph.com/github.com/apache/spark@b012cb722f6105a945e971bdc509f803a90b7419/-/blob/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala?L332:28)). Hence, I think `ExecutorRegistered` will always happen after `sc.schedulerBackend` initialization. 
   
   Do I miss any context? Thank you!



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r942955557


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -77,17 +77,55 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
 
   private[spark] var scheduler: TaskScheduler = null
 
-  // executor ID -> timestamp of when the last heartbeat from this executor was received
+  /**
+   * [SPARK-39984]
+   * Please make sure the intersection between `executorLastSeen` and `waitingList` is an empty set.
+   * If the intersection is not empty, it is possible to never kill the executor until the executor
+   * recovers. When an executor is in both `executorLastSeen` and `waitingList`, the value of
+   * `workerLastHeartbeat` in waitingList may update if the worker sends heartbeats to master
+   * normally.
+   *
+   * `executorLastSeen`:
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from this executor was received
+   *
+   *  `waitingList`: executor ID -> WorkerLastHeartbeat
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from the worker was received
+   *
+   * when driver does not receive any heartbeat from an executor for `executorTimeoutMs` seconds,
+   * the driver will ask master for the last heartbeat from the worker which the executor is running
+   * on.
+   */
   private val executorLastSeen = new HashMap[String, Long]
+  private val waitingList = new HashMap[String, Long]

Review Comment:
   Updated.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r942731292


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +222,131 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms",
+                causedByApp = !sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT))))
+
+          // LocalSchedulerBackend is used locally and only has one single executor
+          case _: LocalSchedulerBackend =>
+
+          case other => throw new UnsupportedOperationException(
+            s"Unknown scheduler backend: ${other.getClass}")
+        }
+      }
+    })
+  }
+
   private def expireDeadHosts(): Unit = {
+  /**
+   * [SC-105641]
+   * Originally, the driver’s HeartbeatReceiver will expire an executor if it does not receive any
+   * heartbeat from the executor for 120 seconds. However, 120 seconds is too long, but we will face
+   * other challenges when we try to lower the timeout threshold. To elaborate, when an executor is
+   * performing full GC, it cannot send/reply any message. Next paragraphs describe the solution to
+   * detect network disconnection between driver and executor in a short time.
+   *
+   * An executor is running on a worker but in different JVMs, and a driver is running on a master
+   * but in different JVMs. Hence, the network connection between driver/executor and master/worker
+   * is the same. Because executor and worker are running on different JVMs, worker can still send
+   * heartbeat to master when executor performs GC.
+   *
+   * For new Heartbeat Receiver, if driver does not receive any heartbeat from the executor for
+   * `executorTimeoutMs` (default: 60s) seconds, HeartbeatReceiver will send a request to master to
+   * ask for the latest heartbeat from the worker which the executor runs on `workerLastHeartbeat`.
+   * HeartbeatReceiver can determine whether the heartbeat loss is caused by network issues or other
+   * issues (e.g. GC). If the heartbeat loss is not caused by network issues, the HeartbeatReceiver
+   * will put the executor into a waitingList rather than expiring it immediately.
+   *
+   * [Note]: Definition of `network issues`
+   * Here, the definition `network issues` is the issues that related to network directly. If the
+   * network is connected, the issues do not included in `network issues`. For example, an
+   * executor's JVM is closed by a problematic task, so the JVM will notify driver that the socket
+   * is closed. If the network is connected, driver will receive the notification and trigger the
+   * function `onDisconnected`. This issue is not a `network issue` because the network is
+   * connected.
+   *
+   * [Warning 1]
+   * Worker will send heartbeats to Master every (conf.get(WORKER_TIMEOUT) * 1000 / 4) milliseconds.
+   * Check deploy/worker/Worker.scala for more details. This new mechanism design is based on the
+   * assumption: (executorTimeoutMs / 2) > (conf.get(WORKER_TIMEOUT) * 1000 / 4).
+   *
+   * [Warning 2]
+   * Not every deployment method schedules driver on master.
+   */
     logTrace("Checking for hosts with no recent heartbeats in HeartbeatReceiver.")
     val now = clock.getTimeMillis()
-    for ((executorId, lastSeenMs) <- executorLastSeen) {
-      if (now - lastSeenMs > executorTimeoutMs) {
-        logWarning(s"Removing executor $executorId with no recent heartbeats: " +
-          s"${now - lastSeenMs} ms exceeds timeout $executorTimeoutMs ms")
-        // Asynchronously kill the executor to avoid blocking the current thread
-        killExecutorThread.submit(new Runnable {
-          override def run(): Unit = Utils.tryLogNonFatalError {
-            // Note: we want to get an executor back after expiring this one,
-            // so do not simply call `sc.killExecutor` here (SPARK-8119)
-            sc.killAndReplaceExecutor(executorId)
-            // SPARK-27348: in case of the executors which are not gracefully shut down,
-            // we should remove lost executors from CoarseGrainedSchedulerBackend manually
-            // here to guarantee two things:
-            // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
-            //    a lost executor instead of waiting for disconnect message
-            // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
-            //    those executors to avoid app hang
-            sc.schedulerBackend match {
-              case backend: CoarseGrainedSchedulerBackend =>
-                backend.driverEndpoint.send(RemoveExecutor(executorId,
-                  ExecutorProcessLost(
-                    s"Executor heartbeat timed out after ${now - lastSeenMs} ms")))
-
-              // LocalSchedulerBackend is used locally and only has one single executor
-              case _: LocalSchedulerBackend =>
-
-              case other => throw new UnsupportedOperationException(
-                s"Unknown scheduler backend: ${other.getClass}")
-            }
+    if (!sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT)) {
+      for ((executorId, lastSeenMs) <- executorLastSeen) {
+        if (now - lastSeenMs > executorTimeoutMs) {
+          killExecutor(executorId, now - lastSeenMs)
+          waitingList.remove(executorId)
+          executorLastSeen.remove(executorId)
+        }
+      }
+    } else {
+      for ((executorId, workerLastHeartbeat) <- waitingList) {
+        if (now - workerLastHeartbeat > executorTimeoutMs / 2) {

Review Comment:
   Add a new config in https://github.com/apache/spark/pull/37411/commits/0c409344877efb481d91d5190f4d801a39910a72.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r942971087


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +222,131 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms",
+                causedByApp = !sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT))))
+
+          // LocalSchedulerBackend is used locally and only has one single executor
+          case _: LocalSchedulerBackend =>
+
+          case other => throw new UnsupportedOperationException(
+            s"Unknown scheduler backend: ${other.getClass}")
+        }
+      }
+    })
+  }
+
   private def expireDeadHosts(): Unit = {
+  /**
+   * [SC-105641]
+   * Originally, the driver’s HeartbeatReceiver will expire an executor if it does not receive any
+   * heartbeat from the executor for 120 seconds. However, 120 seconds is too long, but we will face
+   * other challenges when we try to lower the timeout threshold. To elaborate, when an executor is
+   * performing full GC, it cannot send/reply any message. Next paragraphs describe the solution to
+   * detect network disconnection between driver and executor in a short time.
+   *
+   * An executor is running on a worker but in different JVMs, and a driver is running on a master
+   * but in different JVMs. Hence, the network connection between driver/executor and master/worker
+   * is the same. Because executor and worker are running on different JVMs, worker can still send
+   * heartbeat to master when executor performs GC.
+   *
+   * For new Heartbeat Receiver, if driver does not receive any heartbeat from the executor for
+   * `executorTimeoutMs` (default: 60s) seconds, HeartbeatReceiver will send a request to master to
+   * ask for the latest heartbeat from the worker which the executor runs on `workerLastHeartbeat`.
+   * HeartbeatReceiver can determine whether the heartbeat loss is caused by network issues or other
+   * issues (e.g. GC). If the heartbeat loss is not caused by network issues, the HeartbeatReceiver
+   * will put the executor into a waitingList rather than expiring it immediately.
+   *
+   * [Note]: Definition of `network issues`
+   * Here, the definition `network issues` is the issues that related to network directly. If the
+   * network is connected, the issues do not included in `network issues`. For example, an
+   * executor's JVM is closed by a problematic task, so the JVM will notify driver that the socket
+   * is closed. If the network is connected, driver will receive the notification and trigger the
+   * function `onDisconnected`. This issue is not a `network issue` because the network is
+   * connected.
+   *
+   * [Warning 1]
+   * Worker will send heartbeats to Master every (conf.get(WORKER_TIMEOUT) * 1000 / 4) milliseconds.
+   * Check deploy/worker/Worker.scala for more details. This new mechanism design is based on the
+   * assumption: (executorTimeoutMs / 2) > (conf.get(WORKER_TIMEOUT) * 1000 / 4).
+   *
+   * [Warning 2]
+   * Not every deployment method schedules driver on master.
+   */
     logTrace("Checking for hosts with no recent heartbeats in HeartbeatReceiver.")
     val now = clock.getTimeMillis()
-    for ((executorId, lastSeenMs) <- executorLastSeen) {
-      if (now - lastSeenMs > executorTimeoutMs) {
-        logWarning(s"Removing executor $executorId with no recent heartbeats: " +
-          s"${now - lastSeenMs} ms exceeds timeout $executorTimeoutMs ms")
-        // Asynchronously kill the executor to avoid blocking the current thread
-        killExecutorThread.submit(new Runnable {
-          override def run(): Unit = Utils.tryLogNonFatalError {
-            // Note: we want to get an executor back after expiring this one,
-            // so do not simply call `sc.killExecutor` here (SPARK-8119)
-            sc.killAndReplaceExecutor(executorId)
-            // SPARK-27348: in case of the executors which are not gracefully shut down,
-            // we should remove lost executors from CoarseGrainedSchedulerBackend manually
-            // here to guarantee two things:
-            // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
-            //    a lost executor instead of waiting for disconnect message
-            // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
-            //    those executors to avoid app hang
-            sc.schedulerBackend match {
-              case backend: CoarseGrainedSchedulerBackend =>
-                backend.driverEndpoint.send(RemoveExecutor(executorId,
-                  ExecutorProcessLost(
-                    s"Executor heartbeat timed out after ${now - lastSeenMs} ms")))
-
-              // LocalSchedulerBackend is used locally and only has one single executor
-              case _: LocalSchedulerBackend =>
-
-              case other => throw new UnsupportedOperationException(
-                s"Unknown scheduler backend: ${other.getClass}")
-            }
+    if (!sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT)) {
+      for ((executorId, lastSeenMs) <- executorLastSeen) {
+        if (now - lastSeenMs > executorTimeoutMs) {
+          killExecutor(executorId, now - lastSeenMs)
+          waitingList.remove(executorId)
+          executorLastSeen.remove(executorId)
+        }
+      }
+    } else {
+      for ((executorId, workerLastHeartbeat) <- waitingList) {
+        if (now - workerLastHeartbeat > executorTimeoutMs / 2) {
+          killExecutor(executorId, now - workerLastHeartbeat)
+          waitingList.remove(executorId)
+          executorLastSeen.remove(executorId)
+        }
+      }
+
+      val buf = new ArrayBuffer[String]()
+      for ((executorId, lastSeenMs) <- executorLastSeen) {
+        if (now - lastSeenMs > executorTimeoutMs) {
+          sc.schedulerBackend match {
+            case _: StandaloneSchedulerBackend =>
+              buf += executorId
+            case _ =>
+              killExecutor(executorId, now - lastSeenMs)
+              waitingList.remove(executorId)
+              executorLastSeen.remove(executorId)
+          }
+        }
+      }
+
+      sc.schedulerBackend match {
+        case backend: StandaloneSchedulerBackend =>
+          backend.client.workerLastHeartbeat(sc.applicationId, buf) match {
+            case Some(workerLastHeartbeats) =>
+              for ((executorId, workerLastHeartbeat) <- buf zip workerLastHeartbeats) {
+                if (now - workerLastHeartbeat > executorTimeoutMs / 2) {
+                  val lastSeenMs = executorLastSeen.get(executorId).get
+                  killExecutor(executorId, now - lastSeenMs)
+                  waitingList.remove(executorId)
+                } else {
+                  waitingList(executorId) = workerLastHeartbeat
+                }
+                executorLastSeen.remove(executorId)
+              }
+            case None =>
+              for (executorId <- buf) {
+                val lastSeenMs = executorLastSeen.get(executorId).get
+                killExecutor(executorId, now - lastSeenMs)
+                executorLastSeen.remove(executorId)
+                waitingList.remove(executorId)
+              }

Review Comment:
   Because we discussed this topic below, close this.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r943675732


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +242,137 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            val isStandalone = backend.isInstanceOf[StandaloneSchedulerBackend]
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms",
+                causedByApp = !checkWorkerLastHeartbeat || !isStandalone)))

Review Comment:
   > For case (1), the task will keep getting rerun on a different executors - where it will keep hanging repeatedly - if the hang is due to the user code.
   
   You are right. To prevent infinite retries, we may temporarily set `causedByApp` to true until we have a mechanism to detect hanging task in executor. Does it make sense?
   
   > You are right, (2) cannot be detected - but speculative execution might help with that if it is related to the specific node (if it is related to user code/data, it will hang indefinitely until user intervention).
   
   As I mentioned above, we need a hanging task detector on executor to detect this.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on PR #37411:
URL: https://github.com/apache/spark/pull/37411#issuecomment-1237801841

   I have revisited the configurations in this PR and updated their default values after the discussion with @Ngone51.
   
   (1) initial delay: set initial delay to `executorTimeoutMs` (default: 30s)
   
   ```scala
   override def onStart(): Unit = {
     timeoutCheckingTask = eventLoopThread.scheduleAtFixedRate(
       () => Utils.tryLogNonFatalError { Option(self).foreach(_.ask[Boolean](ExpireDeadHosts)) },
       executorTimeoutMs, checkTimeoutIntervalMs, TimeUnit.MILLISECONDS)
   }
   ```
   
   (2) `checkTimeoutIntervalMs`: the thread in (1) will execute the function `expireDeadHosts` every  `checkTimeoutIntervalMs` (default: 15s).
   ```scala
   private val checkTimeoutIntervalMs = sc.conf.get(Network.NETWORK_TIMEOUT_INTERVAL)
   ```
   
   (3) `executorTimeoutMs`: the default value of `STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT` is optional, and thus the default value is equal to `NETWORK_EXECUTOR_TIMEOUT` (default: 30s).
   ```scala
   private val executorTimeoutMs = sc.conf.get(
     config.STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT
   ).getOrElse(
     sc.conf.get(Network.NETWORK_EXECUTOR_TIMEOUT)
   )
   ```
   
   (4) `expiryCandidatesTimeout`: if `checkWorkerLastHeartbeat` is true, the value will be `HEARTBEAT_EXPIRY_CANDIDATES_TIMEOUT` (default: 90s).
   
   ```scala
   private lazy val expiryCandidatesTimeout = checkWorkerLastHeartbeat match {
     case true =>
       logWarning(s"Worker heartbeat check is enabled. It only works normally when" +
         s"${config.HEARTBEAT_EXPIRY_CANDIDATES_TIMEOUT.key} is larger than worker's" +
         s"heartbeat interval.")
       sc.conf.get(config.HEARTBEAT_EXPIRY_CANDIDATES_TIMEOUT)
     case false => 0
   }
   ```


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r946405148


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +241,120 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            // TODO (SPARK-39984): Update causedByApp when we have a hanging task detector
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms")))
+          // LocalSchedulerBackend is used locally and only has one single executor
+          case _: LocalSchedulerBackend =>
+
+          case other => throw new UnsupportedOperationException(
+            s"Unknown scheduler backend: ${other.getClass}")
+        }
+      }
+    })
+  }
+
+  private def isStandalone(): Boolean = {
+    sc.schedulerBackend match {
+      case backend: StandaloneSchedulerBackend => true
+      case _ => false
+    }
+  }
+
+  private def removeExecutorFromExpiryCandidates(executorId: String): Unit = {
+    if (checkWorkerLastHeartbeat && isStandalone()) {
+      executorExpiryCandidates.remove(executorId)
+    }
+  }
+
   private def expireDeadHosts(): Unit = {
+  /**
+   * [SPARK-39984]
+   * The driver’s HeartbeatReceiver will expire an executor if it does not receive any heartbeat
+   * from the executor for `executorTimeoutMs` (default 120s) seconds. However, lowering from 120
+   * seconds has other challenges. For example: when executor is performing full GC, it cannot
+   * send/reply any message for tens of seconds (based on your environment). Hence,
+   * HeartbeatReceiver cannot whether the heartbeat loss is caused by network issues or other
+   * reasons (e.g. full GC). To address this, we designed a new Heartbeat Receiver mechanism for
+   * standalone deployments.
+   *
+   * For standalone deployments:
+   * If driver does not receive any heartbeat from the executor for `executorTimeoutMs` seconds,
+   * HeartbeatReceiver will send a request to master to ask for the latest heartbeat from the
+   * worker which the executor runs on. HeartbeatReceiver can determine whether the heartbeat loss
+   * is caused by network issues or other issues (e.g. GC). If the heartbeat loss is not caused by
+   * network issues, the HeartbeatReceiver will put the executor into `executorExpiryCandidates`
+   * rather than expiring it immediately.
+   */
     logTrace("Checking for hosts with no recent heartbeats in HeartbeatReceiver.")
+    logWarning(s"Keep `expiryCandidatesTimeout` larger than `HEARTBEAT_MILLIS` in" +

Review Comment:
   Updated https://github.com/apache/spark/pull/37411/commits/dfe31981e2f5627c0033461aa002330808b5a6d7



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

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

   Can one of the admins verify this patch?


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r942074473


##########
core/src/main/scala/org/apache/spark/internal/config/Network.scala:
##########
@@ -49,7 +49,13 @@ private[spark] object Network {
     ConfigBuilder("spark.network.timeoutInterval")
       .version("1.3.2")
       .timeConf(TimeUnit.MILLISECONDS)
-      .createWithDefaultString(STORAGE_BLOCKMANAGER_TIMEOUTINTERVAL.defaultValueString)
+      .createWithDefaultString("15s")

Review Comment:
   Reverted.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] mridulm commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r938566024


##########
core/src/main/scala/org/apache/spark/internal/config/package.scala:
##########
@@ -2398,4 +2398,11 @@ package object config {
       .version("3.3.0")
       .intConf
       .createWithDefault(5)
+
+  private[spark] val HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT =
+    ConfigBuilder("spark.driver.heartbeat.checkWorkerLastHeartbeat")
+      .internal()
+      .version("3.4.0")
+      .booleanConf
+      .createWithDefault(true)

Review Comment:
   Default this to `false`



##########
core/src/main/scala/org/apache/spark/internal/config/Network.scala:
##########
@@ -49,7 +49,13 @@ private[spark] object Network {
     ConfigBuilder("spark.network.timeoutInterval")
       .version("1.3.2")
       .timeConf(TimeUnit.MILLISECONDS)
-      .createWithDefaultString(STORAGE_BLOCKMANAGER_TIMEOUTINTERVAL.defaultValueString)
+      .createWithDefaultString("15s")
+
+  private[spark] val NETWORK_EXECUTOR_TIMEOUT =
+    ConfigBuilder("spark.network.executorTimeout")
+      .version("1.3.0")

Review Comment:
   `1.3.0` -> `3.4.0`



##########
core/src/main/scala/org/apache/spark/internal/config/Network.scala:
##########
@@ -49,7 +49,13 @@ private[spark] object Network {
     ConfigBuilder("spark.network.timeoutInterval")
       .version("1.3.2")
       .timeConf(TimeUnit.MILLISECONDS)
-      .createWithDefaultString(STORAGE_BLOCKMANAGER_TIMEOUTINTERVAL.defaultValueString)
+      .createWithDefaultString("15s")
+
+  private[spark] val NETWORK_EXECUTOR_TIMEOUT =
+    ConfigBuilder("spark.network.executorTimeout")
+      .version("1.3.0")
+      .timeConf(TimeUnit.MILLISECONDS)
+      .createWithDefaultString("60s")

Review Comment:
   fallback to `NETWORK_TIMEOUT` to preserve existing behavior.



##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +222,131 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms",
+                causedByApp = !sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT))))
+
+          // LocalSchedulerBackend is used locally and only has one single executor
+          case _: LocalSchedulerBackend =>
+
+          case other => throw new UnsupportedOperationException(
+            s"Unknown scheduler backend: ${other.getClass}")
+        }
+      }
+    })
+  }
+
   private def expireDeadHosts(): Unit = {
+  /**
+   * [SC-105641]
+   * Originally, the driver’s HeartbeatReceiver will expire an executor if it does not receive any
+   * heartbeat from the executor for 120 seconds. However, 120 seconds is too long, but we will face
+   * other challenges when we try to lower the timeout threshold. To elaborate, when an executor is
+   * performing full GC, it cannot send/reply any message. Next paragraphs describe the solution to
+   * detect network disconnection between driver and executor in a short time.
+   *
+   * An executor is running on a worker but in different JVMs, and a driver is running on a master
+   * but in different JVMs. Hence, the network connection between driver/executor and master/worker
+   * is the same. Because executor and worker are running on different JVMs, worker can still send
+   * heartbeat to master when executor performs GC.
+   *
+   * For new Heartbeat Receiver, if driver does not receive any heartbeat from the executor for
+   * `executorTimeoutMs` (default: 60s) seconds, HeartbeatReceiver will send a request to master to
+   * ask for the latest heartbeat from the worker which the executor runs on `workerLastHeartbeat`.
+   * HeartbeatReceiver can determine whether the heartbeat loss is caused by network issues or other
+   * issues (e.g. GC). If the heartbeat loss is not caused by network issues, the HeartbeatReceiver
+   * will put the executor into a waitingList rather than expiring it immediately.
+   *
+   * [Note]: Definition of `network issues`
+   * Here, the definition `network issues` is the issues that related to network directly. If the
+   * network is connected, the issues do not included in `network issues`. For example, an
+   * executor's JVM is closed by a problematic task, so the JVM will notify driver that the socket
+   * is closed. If the network is connected, driver will receive the notification and trigger the
+   * function `onDisconnected`. This issue is not a `network issue` because the network is
+   * connected.
+   *
+   * [Warning 1]
+   * Worker will send heartbeats to Master every (conf.get(WORKER_TIMEOUT) * 1000 / 4) milliseconds.
+   * Check deploy/worker/Worker.scala for more details. This new mechanism design is based on the
+   * assumption: (executorTimeoutMs / 2) > (conf.get(WORKER_TIMEOUT) * 1000 / 4).
+   *
+   * [Warning 2]
+   * Not every deployment method schedules driver on master.
+   */
     logTrace("Checking for hosts with no recent heartbeats in HeartbeatReceiver.")
     val now = clock.getTimeMillis()
-    for ((executorId, lastSeenMs) <- executorLastSeen) {
-      if (now - lastSeenMs > executorTimeoutMs) {
-        logWarning(s"Removing executor $executorId with no recent heartbeats: " +
-          s"${now - lastSeenMs} ms exceeds timeout $executorTimeoutMs ms")
-        // Asynchronously kill the executor to avoid blocking the current thread
-        killExecutorThread.submit(new Runnable {
-          override def run(): Unit = Utils.tryLogNonFatalError {
-            // Note: we want to get an executor back after expiring this one,
-            // so do not simply call `sc.killExecutor` here (SPARK-8119)
-            sc.killAndReplaceExecutor(executorId)
-            // SPARK-27348: in case of the executors which are not gracefully shut down,
-            // we should remove lost executors from CoarseGrainedSchedulerBackend manually
-            // here to guarantee two things:
-            // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
-            //    a lost executor instead of waiting for disconnect message
-            // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
-            //    those executors to avoid app hang
-            sc.schedulerBackend match {
-              case backend: CoarseGrainedSchedulerBackend =>
-                backend.driverEndpoint.send(RemoveExecutor(executorId,
-                  ExecutorProcessLost(
-                    s"Executor heartbeat timed out after ${now - lastSeenMs} ms")))
-
-              // LocalSchedulerBackend is used locally and only has one single executor
-              case _: LocalSchedulerBackend =>
-
-              case other => throw new UnsupportedOperationException(
-                s"Unknown scheduler backend: ${other.getClass}")
-            }
+    if (!sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT)) {
+      for ((executorId, lastSeenMs) <- executorLastSeen) {
+        if (now - lastSeenMs > executorTimeoutMs) {
+          killExecutor(executorId, now - lastSeenMs)
+          waitingList.remove(executorId)
+          executorLastSeen.remove(executorId)
+        }
+      }
+    } else {
+      for ((executorId, workerLastHeartbeat) <- waitingList) {
+        if (now - workerLastHeartbeat > executorTimeoutMs / 2) {
+          killExecutor(executorId, now - workerLastHeartbeat)
+          waitingList.remove(executorId)
+          executorLastSeen.remove(executorId)
+        }
+      }
+
+      val buf = new ArrayBuffer[String]()
+      for ((executorId, lastSeenMs) <- executorLastSeen) {
+        if (now - lastSeenMs > executorTimeoutMs) {
+          sc.schedulerBackend match {
+            case _: StandaloneSchedulerBackend =>
+              buf += executorId
+            case _ =>
+              killExecutor(executorId, now - lastSeenMs)
+              waitingList.remove(executorId)
+              executorLastSeen.remove(executorId)
+          }
+        }
+      }
+
+      sc.schedulerBackend match {
+        case backend: StandaloneSchedulerBackend =>
+          backend.client.workerLastHeartbeat(sc.applicationId, buf) match {
+            case Some(workerLastHeartbeats) =>
+              for ((executorId, workerLastHeartbeat) <- buf zip workerLastHeartbeats) {
+                if (now - workerLastHeartbeat > executorTimeoutMs / 2) {
+                  val lastSeenMs = executorLastSeen.get(executorId).get
+                  killExecutor(executorId, now - lastSeenMs)
+                  waitingList.remove(executorId)
+                } else {
+                  waitingList(executorId) = workerLastHeartbeat
+                }
+                executorLastSeen.remove(executorId)
+              }
+            case None =>
+              for (executorId <- buf) {
+                val lastSeenMs = executorLastSeen.get(executorId).get
+                killExecutor(executorId, now - lastSeenMs)
+                executorLastSeen.remove(executorId)
+                waitingList.remove(executorId)
+              }

Review Comment:
   Looks like the `if` condition should be `!sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT) && sc.schedulerBackend isInstanceOf StandaloneSchedulerBackend` ?
   
   



##########
core/src/main/scala/org/apache/spark/internal/config/Network.scala:
##########
@@ -49,7 +49,13 @@ private[spark] object Network {
     ConfigBuilder("spark.network.timeoutInterval")
       .version("1.3.2")
       .timeConf(TimeUnit.MILLISECONDS)
-      .createWithDefaultString(STORAGE_BLOCKMANAGER_TIMEOUTINTERVAL.defaultValueString)
+      .createWithDefaultString("15s")

Review Comment:
   Revert ?



##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -77,12 +77,32 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
 
   private[spark] var scheduler: TaskScheduler = null
 
-  // executor ID -> timestamp of when the last heartbeat from this executor was received
+  /**
+   * [SC-105641]
+   * Please make sure the intersection between `executorLastSeen` and `waitingList` is an empty set.
+   * If the intersection is not empty, it is possible to never kill the executor until the executor
+   * recovers. When an executor is in both `executorLastSeen` and `waitingList`, the value of
+   * `workerLastHeartbeat` in waitingList may update if the worker sends heartbeats to master
+   * normally.
+   *
+   * `executorLastSeen`:
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from this executor was received
+   *
+   *  `waitingList`: executor ID -> WorkerLastHeartbeat
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from the worker was received
+   *
+   * when driver does not receive any heartbeat from an executor for `executorTimeoutMs` seconds,
+   * the driver will ask master for the last heartbeat from the worker which the executor is running
+   * on.
+   */
   private val executorLastSeen = new HashMap[String, Long]
+  private val waitingList = new HashMap[String, Long]

Review Comment:
   We need `waitingList` to be updated/maintained only when `HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT` is enabled.



##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +222,131 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms",
+                causedByApp = !sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT))))
+
+          // LocalSchedulerBackend is used locally and only has one single executor
+          case _: LocalSchedulerBackend =>
+
+          case other => throw new UnsupportedOperationException(
+            s"Unknown scheduler backend: ${other.getClass}")
+        }
+      }
+    })
+  }
+
   private def expireDeadHosts(): Unit = {
+  /**
+   * [SC-105641]
+   * Originally, the driver’s HeartbeatReceiver will expire an executor if it does not receive any
+   * heartbeat from the executor for 120 seconds. However, 120 seconds is too long, but we will face
+   * other challenges when we try to lower the timeout threshold. To elaborate, when an executor is
+   * performing full GC, it cannot send/reply any message. Next paragraphs describe the solution to
+   * detect network disconnection between driver and executor in a short time.
+   *
+   * An executor is running on a worker but in different JVMs, and a driver is running on a master
+   * but in different JVMs. Hence, the network connection between driver/executor and master/worker
+   * is the same. Because executor and worker are running on different JVMs, worker can still send
+   * heartbeat to master when executor performs GC.
+   *
+   * For new Heartbeat Receiver, if driver does not receive any heartbeat from the executor for
+   * `executorTimeoutMs` (default: 60s) seconds, HeartbeatReceiver will send a request to master to
+   * ask for the latest heartbeat from the worker which the executor runs on `workerLastHeartbeat`.
+   * HeartbeatReceiver can determine whether the heartbeat loss is caused by network issues or other
+   * issues (e.g. GC). If the heartbeat loss is not caused by network issues, the HeartbeatReceiver
+   * will put the executor into a waitingList rather than expiring it immediately.
+   *
+   * [Note]: Definition of `network issues`
+   * Here, the definition `network issues` is the issues that related to network directly. If the
+   * network is connected, the issues do not included in `network issues`. For example, an
+   * executor's JVM is closed by a problematic task, so the JVM will notify driver that the socket
+   * is closed. If the network is connected, driver will receive the notification and trigger the
+   * function `onDisconnected`. This issue is not a `network issue` because the network is
+   * connected.
+   *
+   * [Warning 1]
+   * Worker will send heartbeats to Master every (conf.get(WORKER_TIMEOUT) * 1000 / 4) milliseconds.
+   * Check deploy/worker/Worker.scala for more details. This new mechanism design is based on the
+   * assumption: (executorTimeoutMs / 2) > (conf.get(WORKER_TIMEOUT) * 1000 / 4).
+   *
+   * [Warning 2]
+   * Not every deployment method schedules driver on master.
+   */
     logTrace("Checking for hosts with no recent heartbeats in HeartbeatReceiver.")
     val now = clock.getTimeMillis()
-    for ((executorId, lastSeenMs) <- executorLastSeen) {
-      if (now - lastSeenMs > executorTimeoutMs) {
-        logWarning(s"Removing executor $executorId with no recent heartbeats: " +
-          s"${now - lastSeenMs} ms exceeds timeout $executorTimeoutMs ms")
-        // Asynchronously kill the executor to avoid blocking the current thread
-        killExecutorThread.submit(new Runnable {
-          override def run(): Unit = Utils.tryLogNonFatalError {
-            // Note: we want to get an executor back after expiring this one,
-            // so do not simply call `sc.killExecutor` here (SPARK-8119)
-            sc.killAndReplaceExecutor(executorId)
-            // SPARK-27348: in case of the executors which are not gracefully shut down,
-            // we should remove lost executors from CoarseGrainedSchedulerBackend manually
-            // here to guarantee two things:
-            // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
-            //    a lost executor instead of waiting for disconnect message
-            // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
-            //    those executors to avoid app hang
-            sc.schedulerBackend match {
-              case backend: CoarseGrainedSchedulerBackend =>
-                backend.driverEndpoint.send(RemoveExecutor(executorId,
-                  ExecutorProcessLost(
-                    s"Executor heartbeat timed out after ${now - lastSeenMs} ms")))
-
-              // LocalSchedulerBackend is used locally and only has one single executor
-              case _: LocalSchedulerBackend =>
-
-              case other => throw new UnsupportedOperationException(
-                s"Unknown scheduler backend: ${other.getClass}")
-            }
+    if (!sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT)) {

Review Comment:
   Pull this out as an instance variable



##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +222,131 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms",
+                causedByApp = !sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT))))
+
+          // LocalSchedulerBackend is used locally and only has one single executor
+          case _: LocalSchedulerBackend =>
+
+          case other => throw new UnsupportedOperationException(
+            s"Unknown scheduler backend: ${other.getClass}")
+        }
+      }
+    })
+  }
+
   private def expireDeadHosts(): Unit = {
+  /**
+   * [SC-105641]
+   * Originally, the driver’s HeartbeatReceiver will expire an executor if it does not receive any
+   * heartbeat from the executor for 120 seconds. However, 120 seconds is too long, but we will face
+   * other challenges when we try to lower the timeout threshold. To elaborate, when an executor is
+   * performing full GC, it cannot send/reply any message. Next paragraphs describe the solution to
+   * detect network disconnection between driver and executor in a short time.
+   *
+   * An executor is running on a worker but in different JVMs, and a driver is running on a master
+   * but in different JVMs. Hence, the network connection between driver/executor and master/worker
+   * is the same. Because executor and worker are running on different JVMs, worker can still send
+   * heartbeat to master when executor performs GC.
+   *
+   * For new Heartbeat Receiver, if driver does not receive any heartbeat from the executor for
+   * `executorTimeoutMs` (default: 60s) seconds, HeartbeatReceiver will send a request to master to
+   * ask for the latest heartbeat from the worker which the executor runs on `workerLastHeartbeat`.
+   * HeartbeatReceiver can determine whether the heartbeat loss is caused by network issues or other
+   * issues (e.g. GC). If the heartbeat loss is not caused by network issues, the HeartbeatReceiver
+   * will put the executor into a waitingList rather than expiring it immediately.
+   *
+   * [Note]: Definition of `network issues`
+   * Here, the definition `network issues` is the issues that related to network directly. If the
+   * network is connected, the issues do not included in `network issues`. For example, an
+   * executor's JVM is closed by a problematic task, so the JVM will notify driver that the socket
+   * is closed. If the network is connected, driver will receive the notification and trigger the
+   * function `onDisconnected`. This issue is not a `network issue` because the network is
+   * connected.
+   *
+   * [Warning 1]
+   * Worker will send heartbeats to Master every (conf.get(WORKER_TIMEOUT) * 1000 / 4) milliseconds.
+   * Check deploy/worker/Worker.scala for more details. This new mechanism design is based on the
+   * assumption: (executorTimeoutMs / 2) > (conf.get(WORKER_TIMEOUT) * 1000 / 4).
+   *
+   * [Warning 2]
+   * Not every deployment method schedules driver on master.
+   */
     logTrace("Checking for hosts with no recent heartbeats in HeartbeatReceiver.")
     val now = clock.getTimeMillis()
-    for ((executorId, lastSeenMs) <- executorLastSeen) {
-      if (now - lastSeenMs > executorTimeoutMs) {
-        logWarning(s"Removing executor $executorId with no recent heartbeats: " +
-          s"${now - lastSeenMs} ms exceeds timeout $executorTimeoutMs ms")
-        // Asynchronously kill the executor to avoid blocking the current thread
-        killExecutorThread.submit(new Runnable {
-          override def run(): Unit = Utils.tryLogNonFatalError {
-            // Note: we want to get an executor back after expiring this one,
-            // so do not simply call `sc.killExecutor` here (SPARK-8119)
-            sc.killAndReplaceExecutor(executorId)
-            // SPARK-27348: in case of the executors which are not gracefully shut down,
-            // we should remove lost executors from CoarseGrainedSchedulerBackend manually
-            // here to guarantee two things:
-            // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
-            //    a lost executor instead of waiting for disconnect message
-            // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
-            //    those executors to avoid app hang
-            sc.schedulerBackend match {
-              case backend: CoarseGrainedSchedulerBackend =>
-                backend.driverEndpoint.send(RemoveExecutor(executorId,
-                  ExecutorProcessLost(
-                    s"Executor heartbeat timed out after ${now - lastSeenMs} ms")))
-
-              // LocalSchedulerBackend is used locally and only has one single executor
-              case _: LocalSchedulerBackend =>
-
-              case other => throw new UnsupportedOperationException(
-                s"Unknown scheduler backend: ${other.getClass}")
-            }
+    if (!sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT)) {
+      for ((executorId, lastSeenMs) <- executorLastSeen) {
+        if (now - lastSeenMs > executorTimeoutMs) {
+          killExecutor(executorId, now - lastSeenMs)
+          waitingList.remove(executorId)
+          executorLastSeen.remove(executorId)
+        }
+      }
+    } else {
+      for ((executorId, workerLastHeartbeat) <- waitingList) {
+        if (now - workerLastHeartbeat > executorTimeoutMs / 2) {

Review Comment:
   why `executorTimeoutMs / 2` ?



##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +222,131 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms",
+                causedByApp = !sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT))))
+
+          // LocalSchedulerBackend is used locally and only has one single executor
+          case _: LocalSchedulerBackend =>
+
+          case other => throw new UnsupportedOperationException(
+            s"Unknown scheduler backend: ${other.getClass}")
+        }
+      }
+    })
+  }
+
   private def expireDeadHosts(): Unit = {
+  /**
+   * [SC-105641]
+   * Originally, the driver’s HeartbeatReceiver will expire an executor if it does not receive any
+   * heartbeat from the executor for 120 seconds. However, 120 seconds is too long, but we will face
+   * other challenges when we try to lower the timeout threshold. To elaborate, when an executor is
+   * performing full GC, it cannot send/reply any message. Next paragraphs describe the solution to
+   * detect network disconnection between driver and executor in a short time.
+   *
+   * An executor is running on a worker but in different JVMs, and a driver is running on a master
+   * but in different JVMs. Hence, the network connection between driver/executor and master/worker
+   * is the same. Because executor and worker are running on different JVMs, worker can still send
+   * heartbeat to master when executor performs GC.
+   *
+   * For new Heartbeat Receiver, if driver does not receive any heartbeat from the executor for
+   * `executorTimeoutMs` (default: 60s) seconds, HeartbeatReceiver will send a request to master to
+   * ask for the latest heartbeat from the worker which the executor runs on `workerLastHeartbeat`.
+   * HeartbeatReceiver can determine whether the heartbeat loss is caused by network issues or other
+   * issues (e.g. GC). If the heartbeat loss is not caused by network issues, the HeartbeatReceiver
+   * will put the executor into a waitingList rather than expiring it immediately.
+   *
+   * [Note]: Definition of `network issues`
+   * Here, the definition `network issues` is the issues that related to network directly. If the
+   * network is connected, the issues do not included in `network issues`. For example, an
+   * executor's JVM is closed by a problematic task, so the JVM will notify driver that the socket
+   * is closed. If the network is connected, driver will receive the notification and trigger the
+   * function `onDisconnected`. This issue is not a `network issue` because the network is
+   * connected.
+   *
+   * [Warning 1]
+   * Worker will send heartbeats to Master every (conf.get(WORKER_TIMEOUT) * 1000 / 4) milliseconds.
+   * Check deploy/worker/Worker.scala for more details. This new mechanism design is based on the
+   * assumption: (executorTimeoutMs / 2) > (conf.get(WORKER_TIMEOUT) * 1000 / 4).
+   *
+   * [Warning 2]
+   * Not every deployment method schedules driver on master.
+   */
     logTrace("Checking for hosts with no recent heartbeats in HeartbeatReceiver.")
     val now = clock.getTimeMillis()
-    for ((executorId, lastSeenMs) <- executorLastSeen) {
-      if (now - lastSeenMs > executorTimeoutMs) {
-        logWarning(s"Removing executor $executorId with no recent heartbeats: " +
-          s"${now - lastSeenMs} ms exceeds timeout $executorTimeoutMs ms")
-        // Asynchronously kill the executor to avoid blocking the current thread
-        killExecutorThread.submit(new Runnable {
-          override def run(): Unit = Utils.tryLogNonFatalError {
-            // Note: we want to get an executor back after expiring this one,
-            // so do not simply call `sc.killExecutor` here (SPARK-8119)
-            sc.killAndReplaceExecutor(executorId)
-            // SPARK-27348: in case of the executors which are not gracefully shut down,
-            // we should remove lost executors from CoarseGrainedSchedulerBackend manually
-            // here to guarantee two things:
-            // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
-            //    a lost executor instead of waiting for disconnect message
-            // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
-            //    those executors to avoid app hang
-            sc.schedulerBackend match {
-              case backend: CoarseGrainedSchedulerBackend =>
-                backend.driverEndpoint.send(RemoveExecutor(executorId,
-                  ExecutorProcessLost(
-                    s"Executor heartbeat timed out after ${now - lastSeenMs} ms")))
-
-              // LocalSchedulerBackend is used locally and only has one single executor
-              case _: LocalSchedulerBackend =>
-
-              case other => throw new UnsupportedOperationException(
-                s"Unknown scheduler backend: ${other.getClass}")
-            }
+    if (!sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT)) {

Review Comment:
   Pull this out as an instance variable



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r954074380


##########
core/src/main/scala/org/apache/spark/internal/config/package.scala:
##########
@@ -2398,4 +2398,20 @@ package object config {
       .version("3.3.0")
       .intConf
       .createWithDefault(5)
+
+  private[spark] val HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT =
+    ConfigBuilder("spark.driver.heartbeat.checkWorkerLastHeartbeat")
+      .internal()

Review Comment:
   Updated https://github.com/apache/spark/pull/37411/commits/02ef67917ac65a088acfaf20e42fe9ef5b823c04



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] mridulm commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r940865962


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +222,131 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms",
+                causedByApp = !sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT))))
+
+          // LocalSchedulerBackend is used locally and only has one single executor
+          case _: LocalSchedulerBackend =>
+
+          case other => throw new UnsupportedOperationException(
+            s"Unknown scheduler backend: ${other.getClass}")
+        }
+      }
+    })
+  }
+
   private def expireDeadHosts(): Unit = {
+  /**
+   * [SC-105641]
+   * Originally, the driver’s HeartbeatReceiver will expire an executor if it does not receive any
+   * heartbeat from the executor for 120 seconds. However, 120 seconds is too long, but we will face
+   * other challenges when we try to lower the timeout threshold. To elaborate, when an executor is
+   * performing full GC, it cannot send/reply any message. Next paragraphs describe the solution to
+   * detect network disconnection between driver and executor in a short time.
+   *
+   * An executor is running on a worker but in different JVMs, and a driver is running on a master
+   * but in different JVMs. Hence, the network connection between driver/executor and master/worker
+   * is the same. Because executor and worker are running on different JVMs, worker can still send
+   * heartbeat to master when executor performs GC.
+   *
+   * For new Heartbeat Receiver, if driver does not receive any heartbeat from the executor for
+   * `executorTimeoutMs` (default: 60s) seconds, HeartbeatReceiver will send a request to master to
+   * ask for the latest heartbeat from the worker which the executor runs on `workerLastHeartbeat`.
+   * HeartbeatReceiver can determine whether the heartbeat loss is caused by network issues or other
+   * issues (e.g. GC). If the heartbeat loss is not caused by network issues, the HeartbeatReceiver
+   * will put the executor into a waitingList rather than expiring it immediately.
+   *
+   * [Note]: Definition of `network issues`
+   * Here, the definition `network issues` is the issues that related to network directly. If the
+   * network is connected, the issues do not included in `network issues`. For example, an
+   * executor's JVM is closed by a problematic task, so the JVM will notify driver that the socket
+   * is closed. If the network is connected, driver will receive the notification and trigger the
+   * function `onDisconnected`. This issue is not a `network issue` because the network is
+   * connected.
+   *
+   * [Warning 1]
+   * Worker will send heartbeats to Master every (conf.get(WORKER_TIMEOUT) * 1000 / 4) milliseconds.
+   * Check deploy/worker/Worker.scala for more details. This new mechanism design is based on the
+   * assumption: (executorTimeoutMs / 2) > (conf.get(WORKER_TIMEOUT) * 1000 / 4).
+   *
+   * [Warning 2]
+   * Not every deployment method schedules driver on master.
+   */
     logTrace("Checking for hosts with no recent heartbeats in HeartbeatReceiver.")
     val now = clock.getTimeMillis()
-    for ((executorId, lastSeenMs) <- executorLastSeen) {
-      if (now - lastSeenMs > executorTimeoutMs) {
-        logWarning(s"Removing executor $executorId with no recent heartbeats: " +
-          s"${now - lastSeenMs} ms exceeds timeout $executorTimeoutMs ms")
-        // Asynchronously kill the executor to avoid blocking the current thread
-        killExecutorThread.submit(new Runnable {
-          override def run(): Unit = Utils.tryLogNonFatalError {
-            // Note: we want to get an executor back after expiring this one,
-            // so do not simply call `sc.killExecutor` here (SPARK-8119)
-            sc.killAndReplaceExecutor(executorId)
-            // SPARK-27348: in case of the executors which are not gracefully shut down,
-            // we should remove lost executors from CoarseGrainedSchedulerBackend manually
-            // here to guarantee two things:
-            // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
-            //    a lost executor instead of waiting for disconnect message
-            // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
-            //    those executors to avoid app hang
-            sc.schedulerBackend match {
-              case backend: CoarseGrainedSchedulerBackend =>
-                backend.driverEndpoint.send(RemoveExecutor(executorId,
-                  ExecutorProcessLost(
-                    s"Executor heartbeat timed out after ${now - lastSeenMs} ms")))
-
-              // LocalSchedulerBackend is used locally and only has one single executor
-              case _: LocalSchedulerBackend =>
-
-              case other => throw new UnsupportedOperationException(
-                s"Unknown scheduler backend: ${other.getClass}")
-            }
+    if (!sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT)) {
+      for ((executorId, lastSeenMs) <- executorLastSeen) {
+        if (now - lastSeenMs > executorTimeoutMs) {
+          killExecutor(executorId, now - lastSeenMs)
+          waitingList.remove(executorId)
+          executorLastSeen.remove(executorId)
+        }
+      }
+    } else {
+      for ((executorId, workerLastHeartbeat) <- waitingList) {
+        if (now - workerLastHeartbeat > executorTimeoutMs / 2) {

Review Comment:
   Ideally, yes - since they mean different things; though I can see how closely related they are. +CC @Ngone51 thoughts ?
   
   At a minimum, we should document what the config means, and what implications are of setting it in the config's doc.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r946374353


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +241,120 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            // TODO (SPARK-39984): Update causedByApp when we have a hanging task detector
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms")))
+          // LocalSchedulerBackend is used locally and only has one single executor
+          case _: LocalSchedulerBackend =>
+
+          case other => throw new UnsupportedOperationException(
+            s"Unknown scheduler backend: ${other.getClass}")
+        }
+      }
+    })
+  }
+
+  private def isStandalone(): Boolean = {
+    sc.schedulerBackend match {
+      case backend: StandaloneSchedulerBackend => true
+      case _ => false
+    }

Review Comment:
   Hi @Ngone51, thank you for your review! 
   
   Because `schedulerBackend` is possible to be None, I used pattern matching instead of `isInstanceOf`. @mridulm and I have discussed the issues above. I will tag you in these two discussion threads.
   
   Updated:
   I checked the syntax again. The value of `None.isInstanceOf[StandaloneSchedulerBackend]` is false instead of null pointer exception. You are correct. We can use `isInstanceOf` to replace pattern matching.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on PR #37411:
URL: https://github.com/apache/spark/pull/37411#issuecomment-1213540619

   > Btw, any thoughts on this ?
   > 
   > > Are the changes here necessarily only for standalone ? Why not k8s and yarn ?
   > 
   > The changes are specific to standalone - but the idea should be transferable to others as well, right ? Do we want to tackle that here ?
   
   I can take YARN and k8s support as follow-ups of this PR. In my opinion, the scope of this PR will become too large if we tackle YARN and k8s here. Thank you!


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r963377849


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -77,17 +77,61 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
 
   private[spark] var scheduler: TaskScheduler = null
 
-  // executor ID -> timestamp of when the last heartbeat from this executor was received
+  /**
+   * [SPARK-39984]
+   * Please make sure the intersection between `executorLastSeen` and `executorExpiryCandidates` is
+   * an empty set. If the intersection is not empty, it is possible to never kill the executor until
+   * the executor recovers. When an executor is in both `executorLastSeen` and
+   * `executorExpiryCandidates`, the value of `workerLastHeartbeat` in `executorExpiryCandidates`
+   * may update if the worker sends heartbeats to master normally.
+   *
+   * `executorLastSeen`:
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from this executor was received
+   *
+   * `executorExpiryCandidates`: executor ID -> WorkerLastHeartbeat
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from the worker was received
+   *
+   * when driver does not receive any heartbeat from an executor for `executorTimeoutMs` seconds,
+   * the driver will ask master for the last heartbeat from the worker which the executor is running
+   * on.
+   */
   private val executorLastSeen = new HashMap[String, Long]
+  private val executorExpiryCandidates = new HashMap[String, Long]
 
   private val executorTimeoutMs = sc.conf.get(
     config.STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT
-  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s"))
+  ).getOrElse(
+    sc.conf.get(Network.NETWORK_EXECUTOR_TIMEOUT) match {
+      case Some(executorTimeout) => executorTimeout
+      case None => Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s")
+    }
+  )
 
   private val checkTimeoutIntervalMs = sc.conf.get(Network.NETWORK_TIMEOUT_INTERVAL)
 
   private val executorHeartbeatIntervalMs = sc.conf.get(config.EXECUTOR_HEARTBEAT_INTERVAL)
 
+  /**
+   * Currently, [SPARK-39984] is only for StandaloneSchedulerBackend.
+   *
+   * `checkWorkerLastHeartbeat`: A flag to enable two-phase executor timeout.
+   * `expiryCandidatesTimeout`: The timeout used for executorExpiryCandidates.
+   */
+  private val checkWorkerLastHeartbeat = {

Review Comment:
   Updated https://github.com/apache/spark/pull/37411/commits/92629e30410d7ae9741457240c3f1a789f6b042b. 
   
   (1) Declare the variable `checkWorkerLastHeartbeat` as a lazy variable
   (2) Set an initial delay for `ExpireDeadHosts`
   (3) If the scheduler backend has not initialized when `checkWorkerLastHeartbeat` is evaluated, an `UnsupportedOperationException` will be thrown.
   ```scala
   private lazy val checkWorkerLastHeartbeat = sc.schedulerBackend match {
     case _: CoarseGrainedSchedulerBackend =>
       sc.conf.get(config.HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT) &&
         sc.schedulerBackend.isInstanceOf[StandaloneSchedulerBackend]
     case _: LocalSchedulerBackend => false
     case other => throw new UnsupportedOperationException(
       s"Unknown scheduler backend: ${other.getClass}")
   }
   ```



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on PR #37411:
URL: https://github.com/apache/spark/pull/37411#issuecomment-1223561169

   Gentle ping @Ngone51 @mridulm 


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] mridulm commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r940864007


##########
core/src/main/scala/org/apache/spark/internal/config/package.scala:
##########
@@ -2398,4 +2398,11 @@ package object config {
       .version("3.3.0")
       .intConf
       .createWithDefault(5)
+
+  private[spark] val HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT =
+    ConfigBuilder("spark.driver.heartbeat.checkWorkerLastHeartbeat")
+      .internal()
+      .version("3.4.0")
+      .booleanConf
+      .createWithDefault(true)

Review Comment:
   Please update the relevant tests, so that they test with this being `true`/`false` as relevant - instead of a one-time CI/CD pass : so that future evolution of the code will continue to test this feature (with and without the flag)



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r943743451


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +242,137 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            val isStandalone = backend.isInstanceOf[StandaloneSchedulerBackend]
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms",
+                causedByApp = !checkWorkerLastHeartbeat || !isStandalone)))
+
+          // LocalSchedulerBackend is used locally and only has one single executor
+          case _: LocalSchedulerBackend =>
+
+          case other => throw new UnsupportedOperationException(
+            s"Unknown scheduler backend: ${other.getClass}")
+        }
+      }
+    })
+  }
+
+  private def removeExecutorFromWaitingList(executorId: String): Unit = {
+    val isStandalone = sc.schedulerBackend.isInstanceOf[StandaloneSchedulerBackend]
+    if (checkWorkerLastHeartbeat && isStandalone) {
+      waitingList.remove(executorId)
+    }
+  }
+
   private def expireDeadHosts(): Unit = {
+  /**
+   * [SPARK-39984]
+   * Originally, the driver’s HeartbeatReceiver will expire an executor if it does not receive any
+   * heartbeat from the executor for 120 seconds. However, 120 seconds is too long, but we will face
+   * other challenges when we try to lower the timeout threshold. To elaborate, when an executor is
+   * performing full GC, it cannot send/reply any message. Next paragraphs describe the solution to
+   * detect network disconnection between driver and executor in a short time.
+   *
+   * An executor is running on a worker but in different JVMs, and a driver is running on a master
+   * but in different JVMs. Hence, the network connection between driver/executor and master/worker
+   * is the same. Because executor and worker are running on different JVMs, worker can still send
+   * heartbeat to master when executor performs GC.
+   *
+   * For new Heartbeat Receiver, if driver does not receive any heartbeat from the executor for
+   * `executorTimeoutMs` (default: 60s) seconds, HeartbeatReceiver will send a request to master to
+   * ask for the latest heartbeat from the worker which the executor runs on `workerLastHeartbeat`.
+   * HeartbeatReceiver can determine whether the heartbeat loss is caused by network issues or other
+   * issues (e.g. GC). If the heartbeat loss is not caused by network issues, the HeartbeatReceiver
+   * will put the executor into a waitingList rather than expiring it immediately.
+   *
+   * [Note]: Definition of `network issues`
+   * Here, the definition `network issues` is the issues that related to network directly. If the
+   * network is connected, the issues do not included in `network issues`. For example, an
+   * executor's JVM is closed by a problematic task, so the JVM will notify driver that the socket
+   * is closed. If the network is connected, driver will receive the notification and trigger the
+   * function `onDisconnected`. This issue is not a `network issue` because the network is
+   * connected.
+   *
+   * [Warning 1]
+   * Worker will send heartbeats to Master every (conf.get(WORKER_TIMEOUT) * 1000 / 4) milliseconds.
+   * Check deploy/worker/Worker.scala for more details. This new mechanism design is based on the
+   * assumption: `waitingListTimeout` > (conf.get(WORKER_TIMEOUT) * 1000 / 4).
+   *
+   * [Warning 2]
+   * Not every deployment method schedules driver on master.
+   */

Review Comment:
   Added it as a warning, and simplified comments.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] mridulm commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r942841928


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -77,17 +77,55 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
 
   private[spark] var scheduler: TaskScheduler = null
 
-  // executor ID -> timestamp of when the last heartbeat from this executor was received
+  /**
+   * [SPARK-39984]
+   * Please make sure the intersection between `executorLastSeen` and `waitingList` is an empty set.
+   * If the intersection is not empty, it is possible to never kill the executor until the executor
+   * recovers. When an executor is in both `executorLastSeen` and `waitingList`, the value of
+   * `workerLastHeartbeat` in waitingList may update if the worker sends heartbeats to master
+   * normally.
+   *
+   * `executorLastSeen`:
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from this executor was received
+   *
+   *  `waitingList`: executor ID -> WorkerLastHeartbeat
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from the worker was received
+   *
+   * when driver does not receive any heartbeat from an executor for `executorTimeoutMs` seconds,
+   * the driver will ask master for the last heartbeat from the worker which the executor is running
+   * on.
+   */
   private val executorLastSeen = new HashMap[String, Long]
+  private val waitingList = new HashMap[String, Long]

Review Comment:
   Rename from `waitingList` (and related variables/methods in this class) ?
   Something like `executorExpiryCandidates` or better.



##########
core/src/main/scala/org/apache/spark/internal/config/Network.scala:
##########
@@ -51,6 +51,21 @@ private[spark] object Network {
       .timeConf(TimeUnit.MILLISECONDS)
       .createWithDefaultString(STORAGE_BLOCKMANAGER_TIMEOUTINTERVAL.defaultValueString)
 
+  private[spark] val NETWORK_EXECUTOR_TIMEOUT =
+    ConfigBuilder("spark.network.executorTimeout")
+      .version("3.4.0")
+      .timeConf(TimeUnit.SECONDS)
+      .createOptional

Review Comment:
   QQ: Any particular reason for this to be `s` here and `ms` for `HEARTBEAT_WAITINGLIST_TIMEOUT` ?
   Keep both consistent as `ms` ?



##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +242,137 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            val isStandalone = backend.isInstanceOf[StandaloneSchedulerBackend]
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms",
+                causedByApp = !checkWorkerLastHeartbeat || !isStandalone)))

Review Comment:
   QQ: If the executor/jvm hangs (due to interaction with the application code), we wont be able to detect that anymore, right ? Or is that handled in any other way ?



##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -130,8 +170,11 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     case heartbeat @ Heartbeat(executorId, accumUpdates, blockManagerId, executorUpdates) =>
       var reregisterBlockManager = !sc.isStopped
       if (scheduler != null) {
-        if (executorLastSeen.contains(executorId)) {
+        val isStandalone = sc.schedulerBackend.isInstanceOf[StandaloneSchedulerBackend]

Review Comment:
   nit: pull this out as a method or populate as a field when `TaskSchedulerIsSet` is fired ?



##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -77,17 +77,55 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
 
   private[spark] var scheduler: TaskScheduler = null
 
-  // executor ID -> timestamp of when the last heartbeat from this executor was received
+  /**
+   * [SPARK-39984]
+   * Please make sure the intersection between `executorLastSeen` and `waitingList` is an empty set.
+   * If the intersection is not empty, it is possible to never kill the executor until the executor
+   * recovers. When an executor is in both `executorLastSeen` and `waitingList`, the value of
+   * `workerLastHeartbeat` in waitingList may update if the worker sends heartbeats to master
+   * normally.
+   *
+   * `executorLastSeen`:
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from this executor was received
+   *
+   *  `waitingList`: executor ID -> WorkerLastHeartbeat
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from the worker was received
+   *
+   * when driver does not receive any heartbeat from an executor for `executorTimeoutMs` seconds,
+   * the driver will ask master for the last heartbeat from the worker which the executor is running
+   * on.
+   */
   private val executorLastSeen = new HashMap[String, Long]
+  private val waitingList = new HashMap[String, Long]
 
   private val executorTimeoutMs = sc.conf.get(
     config.STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT
-  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s"))
+  ).getOrElse(
+    sc.conf.get(Network.NETWORK_EXECUTOR_TIMEOUT) match {
+      case Some(executorTimeout) => Utils.timeStringAsMs(s"${executorTimeout}s")
+      case None => Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s")
+    }
+  )
 
   private val checkTimeoutIntervalMs = sc.conf.get(Network.NETWORK_TIMEOUT_INTERVAL)
 
   private val executorHeartbeatIntervalMs = sc.conf.get(config.EXECUTOR_HEARTBEAT_INTERVAL)
 
+  /**
+   * Currently, [SPARK-39984] is only for StandaloneSchedulerBackend.
+   *
+   * `checkWorkerLastHeartbeat`: A flag to enable two-phase executor timeout.
+   * `waitingListTimeout`: The timeout used for waitingList.
+   */
+  private val checkWorkerLastHeartbeat = sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT)
+  private val waitingListTimeout = checkWorkerLastHeartbeat match {
+    case true =>
+      sc.conf.get(Network.HEARTBEAT_WAITINGLIST_TIMEOUT).getOrElse(Utils.timeStringAsMs("30s"))
+    case false => Utils.timeStringAsMs("0s")
+  }
+

Review Comment:
   Make the 30s default value in `HEARTBEAT_WAITINGLIST_TIMEOUT` config (use `createWithDefaultString(30s)` there).
   ```suggestion
     private val waitingListTimeout = if (checkWorkerLastHeartbeat) sc.conf.get(Network.HEARTBEAT_WAITINGLIST_TIMEOUT) else 0
   
   ```



##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +242,137 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            val isStandalone = backend.isInstanceOf[StandaloneSchedulerBackend]
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms",
+                causedByApp = !checkWorkerLastHeartbeat || !isStandalone)))
+
+          // LocalSchedulerBackend is used locally and only has one single executor
+          case _: LocalSchedulerBackend =>
+
+          case other => throw new UnsupportedOperationException(
+            s"Unknown scheduler backend: ${other.getClass}")
+        }
+      }
+    })
+  }
+
+  private def removeExecutorFromWaitingList(executorId: String): Unit = {
+    val isStandalone = sc.schedulerBackend.isInstanceOf[StandaloneSchedulerBackend]
+    if (checkWorkerLastHeartbeat && isStandalone) {
+      waitingList.remove(executorId)
+    }
+  }
+
   private def expireDeadHosts(): Unit = {
+  /**
+   * [SPARK-39984]
+   * Originally, the driver’s HeartbeatReceiver will expire an executor if it does not receive any
+   * heartbeat from the executor for 120 seconds. However, 120 seconds is too long, but we will face
+   * other challenges when we try to lower the timeout threshold. To elaborate, when an executor is
+   * performing full GC, it cannot send/reply any message. Next paragraphs describe the solution to
+   * detect network disconnection between driver and executor in a short time.
+   *
+   * An executor is running on a worker but in different JVMs, and a driver is running on a master
+   * but in different JVMs. Hence, the network connection between driver/executor and master/worker
+   * is the same. Because executor and worker are running on different JVMs, worker can still send
+   * heartbeat to master when executor performs GC.
+   *
+   * For new Heartbeat Receiver, if driver does not receive any heartbeat from the executor for
+   * `executorTimeoutMs` (default: 60s) seconds, HeartbeatReceiver will send a request to master to
+   * ask for the latest heartbeat from the worker which the executor runs on `workerLastHeartbeat`.
+   * HeartbeatReceiver can determine whether the heartbeat loss is caused by network issues or other
+   * issues (e.g. GC). If the heartbeat loss is not caused by network issues, the HeartbeatReceiver
+   * will put the executor into a waitingList rather than expiring it immediately.
+   *
+   * [Note]: Definition of `network issues`
+   * Here, the definition `network issues` is the issues that related to network directly. If the
+   * network is connected, the issues do not included in `network issues`. For example, an
+   * executor's JVM is closed by a problematic task, so the JVM will notify driver that the socket
+   * is closed. If the network is connected, driver will receive the notification and trigger the
+   * function `onDisconnected`. This issue is not a `network issue` because the network is
+   * connected.
+   *
+   * [Warning 1]
+   * Worker will send heartbeats to Master every (conf.get(WORKER_TIMEOUT) * 1000 / 4) milliseconds.
+   * Check deploy/worker/Worker.scala for more details. This new mechanism design is based on the
+   * assumption: `waitingListTimeout` > (conf.get(WORKER_TIMEOUT) * 1000 / 4).
+   *
+   * [Warning 2]
+   * Not every deployment method schedules driver on master.
+   */

Review Comment:
   Let us make this comment more concise, and what the state would be after this PR is merged.
   
   Something like
   ```suggestion
     /**
      * [SPARK-39984]
      * The driver’s HeartbeatReceiver will expire an executor if it does not receive any
      * heartbeat from the executor for 120 seconds. However, lowering from 120 seconds has 
      * other challenges. For example: when executor is performing full GC it cannot send/reply any message.
      *
      * For standalone deployments:
      * If driver does not receive any heartbeat from the executor for
      * `executorTimeoutMs` (default: 60s) seconds, HeartbeatReceiver will send a request to master to
      * ask for the latest heartbeat from the worker which the executor runs on.
      * HeartbeatReceiver can determine whether the heartbeat loss is caused by network issues or other
      * issues (e.g. GC). If the heartbeat loss is not caused by network issues, the HeartbeatReceiver
      * will put the executor into a waitingList rather than expiring it immediately.
      */
   ```
   
   Also, let us move this to the configuration of `HEARTBEAT_WAITINGLIST_TIMEOUT`:
   ```
      * assumption: `waitingListTimeout` > (conf.get(WORKER_TIMEOUT) * 1000 / 4).
   ```
   
   If this is a strong requirement, add as warning when fetching the the config above ?



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r946369394


##########
core/src/main/scala/org/apache/spark/internal/config/Network.scala:
##########
@@ -51,6 +51,21 @@ private[spark] object Network {
       .timeConf(TimeUnit.MILLISECONDS)
       .createWithDefaultString(STORAGE_BLOCKMANAGER_TIMEOUTINTERVAL.defaultValueString)
 
+  private[spark] val NETWORK_EXECUTOR_TIMEOUT =
+    ConfigBuilder("spark.network.executorTimeout")
+      .version("3.4.0")
+      .timeConf(TimeUnit.MILLISECONDS)
+      .createOptional
+
+  private[spark] val HEARTBEAT_EXPIRY_CANDIDATES_TIMEOUT =
+    ConfigBuilder("spark.network.expiryCandidatesTimeout")
+      .doc("This config is a timeout used for heartbeat receiver `executorExpiryCandidates`. Be" +
+        "effective only when HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT is enabled. See" +
+        "[SPARK-39984] for more details")
+      .version("3.4.0")
+      .timeConf(TimeUnit.MILLISECONDS)
+      .createWithDefaultString("30s")

Review Comment:
   Updated https://github.com/apache/spark/pull/37411/commits/e14853527434b7d625f0de05800314d00aaa5fc2



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r946373638


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -77,17 +77,44 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
 
   private[spark] var scheduler: TaskScheduler = null
 
-  // executor ID -> timestamp of when the last heartbeat from this executor was received
+  /**
+   * [SPARK-39984]
+   * Please make sure the intersection between `executorLastSeen` and `waitingList` is an empty set.
+   * If the intersection is not empty, it is possible to never kill the executor until the executor
+   * recovers. When an executor is in both `executorLastSeen` and `waitingList`, the value of
+   * `workerLastHeartbeat` in waitingList may update if the worker sends heartbeats to master
+   * normally.
+   *
+   * `executorLastSeen`:
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from this executor was received
+   *
+   *  `waitingList`: executor ID -> WorkerLastHeartbeat
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from the worker was received
+   *
+   * when driver does not receive any heartbeat from an executor for `executorTimeoutMs` seconds,
+   * the driver will ask master for the last heartbeat from the worker which the executor is running
+   * on.
+   */
   private val executorLastSeen = new HashMap[String, Long]
+  private val waitingList = new HashMap[String, Long]
 
   private val executorTimeoutMs = sc.conf.get(
     config.STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT
-  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s"))
+  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_EXECUTOR_TIMEOUT)}s"))
 
   private val checkTimeoutIntervalMs = sc.conf.get(Network.NETWORK_TIMEOUT_INTERVAL)
 
   private val executorHeartbeatIntervalMs = sc.conf.get(config.EXECUTOR_HEARTBEAT_INTERVAL)
 
+  /**
+   * Currently, [SPARK-39984] is only for StandaloneSchedulerBackend.
+   */
+  private val checkWorkerLastHeartbeat =
+    sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT) &&
+      sc.schedulerBackend.isInstanceOf[StandaloneSchedulerBackend]

Review Comment:
   @Ngone51 Here is the other discussion about the lack of scheduler backend.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] mridulm commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r942958914


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -77,17 +77,44 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
 
   private[spark] var scheduler: TaskScheduler = null
 
-  // executor ID -> timestamp of when the last heartbeat from this executor was received
+  /**
+   * [SPARK-39984]
+   * Please make sure the intersection between `executorLastSeen` and `waitingList` is an empty set.
+   * If the intersection is not empty, it is possible to never kill the executor until the executor
+   * recovers. When an executor is in both `executorLastSeen` and `waitingList`, the value of
+   * `workerLastHeartbeat` in waitingList may update if the worker sends heartbeats to master
+   * normally.
+   *
+   * `executorLastSeen`:
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from this executor was received
+   *
+   *  `waitingList`: executor ID -> WorkerLastHeartbeat
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from the worker was received
+   *
+   * when driver does not receive any heartbeat from an executor for `executorTimeoutMs` seconds,
+   * the driver will ask master for the last heartbeat from the worker which the executor is running
+   * on.
+   */
   private val executorLastSeen = new HashMap[String, Long]
+  private val waitingList = new HashMap[String, Long]
 
   private val executorTimeoutMs = sc.conf.get(
     config.STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT
-  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s"))
+  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_EXECUTOR_TIMEOUT)}s"))
 
   private val checkTimeoutIntervalMs = sc.conf.get(Network.NETWORK_TIMEOUT_INTERVAL)
 
   private val executorHeartbeatIntervalMs = sc.conf.get(config.EXECUTOR_HEARTBEAT_INTERVAL)
 
+  /**
+   * Currently, [SPARK-39984] is only for StandaloneSchedulerBackend.
+   */
+  private val checkWorkerLastHeartbeat =
+    sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT) &&
+      sc.schedulerBackend.isInstanceOf[StandaloneSchedulerBackend]

Review Comment:
   This code has evolved a lot, so it is possible that the check in handling `Heartbeat` is no longer valid (the `null` check there is equally not likely given your comment above) - having said that, I have not looked at all corner cases, so would be careful.
   I will let @Ngone51 comment more as part of his review.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r942963188


##########
core/src/main/scala/org/apache/spark/internal/config/Network.scala:
##########
@@ -51,6 +51,21 @@ private[spark] object Network {
       .timeConf(TimeUnit.MILLISECONDS)
       .createWithDefaultString(STORAGE_BLOCKMANAGER_TIMEOUTINTERVAL.defaultValueString)
 
+  private[spark] val NETWORK_EXECUTOR_TIMEOUT =
+    ConfigBuilder("spark.network.executorTimeout")
+      .version("3.4.0")
+      .timeConf(TimeUnit.SECONDS)
+      .createOptional

Review Comment:
   Update to MILLISECONDS.
   



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r943116859


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +242,137 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            val isStandalone = backend.isInstanceOf[StandaloneSchedulerBackend]
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms",
+                causedByApp = !checkWorkerLastHeartbeat || !isStandalone)))
+
+          // LocalSchedulerBackend is used locally and only has one single executor
+          case _: LocalSchedulerBackend =>
+
+          case other => throw new UnsupportedOperationException(
+            s"Unknown scheduler backend: ${other.getClass}")
+        }
+      }
+    })
+  }
+
+  private def removeExecutorFromWaitingList(executorId: String): Unit = {
+    val isStandalone = sc.schedulerBackend.isInstanceOf[StandaloneSchedulerBackend]
+    if (checkWorkerLastHeartbeat && isStandalone) {
+      waitingList.remove(executorId)
+    }
+  }
+
   private def expireDeadHosts(): Unit = {
+  /**
+   * [SPARK-39984]
+   * Originally, the driver’s HeartbeatReceiver will expire an executor if it does not receive any
+   * heartbeat from the executor for 120 seconds. However, 120 seconds is too long, but we will face
+   * other challenges when we try to lower the timeout threshold. To elaborate, when an executor is
+   * performing full GC, it cannot send/reply any message. Next paragraphs describe the solution to
+   * detect network disconnection between driver and executor in a short time.
+   *
+   * An executor is running on a worker but in different JVMs, and a driver is running on a master
+   * but in different JVMs. Hence, the network connection between driver/executor and master/worker
+   * is the same. Because executor and worker are running on different JVMs, worker can still send
+   * heartbeat to master when executor performs GC.
+   *
+   * For new Heartbeat Receiver, if driver does not receive any heartbeat from the executor for
+   * `executorTimeoutMs` (default: 60s) seconds, HeartbeatReceiver will send a request to master to
+   * ask for the latest heartbeat from the worker which the executor runs on `workerLastHeartbeat`.
+   * HeartbeatReceiver can determine whether the heartbeat loss is caused by network issues or other
+   * issues (e.g. GC). If the heartbeat loss is not caused by network issues, the HeartbeatReceiver
+   * will put the executor into a waitingList rather than expiring it immediately.
+   *
+   * [Note]: Definition of `network issues`
+   * Here, the definition `network issues` is the issues that related to network directly. If the
+   * network is connected, the issues do not included in `network issues`. For example, an
+   * executor's JVM is closed by a problematic task, so the JVM will notify driver that the socket
+   * is closed. If the network is connected, driver will receive the notification and trigger the
+   * function `onDisconnected`. This issue is not a `network issue` because the network is
+   * connected.
+   *
+   * [Warning 1]
+   * Worker will send heartbeats to Master every (conf.get(WORKER_TIMEOUT) * 1000 / 4) milliseconds.
+   * Check deploy/worker/Worker.scala for more details. This new mechanism design is based on the
+   * assumption: `waitingListTimeout` > (conf.get(WORKER_TIMEOUT) * 1000 / 4).
+   *
+   * [Warning 2]
+   * Not every deployment method schedules driver on master.
+   */

Review Comment:
   (Note: `waitingListTimeout` is renamed to `expiryCandidatesTimeout`)
   ```
      * assumption: waitingListTimeout > (conf.get(WORKER_TIMEOUT) * 1000 / 4).
   ```
   
   This is not a strong requirement, but it will make more sense if we fulfill this requirement. If we fulfill requirement, we can know whether master lost any heartbeat from the worker or not.
   
   I can add it as a warning, but I am not sure what is a "warning". Do you mean `logWarning`? Thank you!



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on PR #37411:
URL: https://github.com/apache/spark/pull/37411#issuecomment-1205917792

   cc. @Ngone51 @jiangxb1987 


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r942065632


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +222,131 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms",
+                causedByApp = !sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT))))
+
+          // LocalSchedulerBackend is used locally and only has one single executor
+          case _: LocalSchedulerBackend =>
+
+          case other => throw new UnsupportedOperationException(
+            s"Unknown scheduler backend: ${other.getClass}")
+        }
+      }
+    })
+  }
+
   private def expireDeadHosts(): Unit = {
+  /**
+   * [SC-105641]
+   * Originally, the driver’s HeartbeatReceiver will expire an executor if it does not receive any
+   * heartbeat from the executor for 120 seconds. However, 120 seconds is too long, but we will face
+   * other challenges when we try to lower the timeout threshold. To elaborate, when an executor is
+   * performing full GC, it cannot send/reply any message. Next paragraphs describe the solution to
+   * detect network disconnection between driver and executor in a short time.
+   *
+   * An executor is running on a worker but in different JVMs, and a driver is running on a master
+   * but in different JVMs. Hence, the network connection between driver/executor and master/worker
+   * is the same. Because executor and worker are running on different JVMs, worker can still send
+   * heartbeat to master when executor performs GC.
+   *
+   * For new Heartbeat Receiver, if driver does not receive any heartbeat from the executor for
+   * `executorTimeoutMs` (default: 60s) seconds, HeartbeatReceiver will send a request to master to
+   * ask for the latest heartbeat from the worker which the executor runs on `workerLastHeartbeat`.
+   * HeartbeatReceiver can determine whether the heartbeat loss is caused by network issues or other
+   * issues (e.g. GC). If the heartbeat loss is not caused by network issues, the HeartbeatReceiver
+   * will put the executor into a waitingList rather than expiring it immediately.
+   *
+   * [Note]: Definition of `network issues`
+   * Here, the definition `network issues` is the issues that related to network directly. If the
+   * network is connected, the issues do not included in `network issues`. For example, an
+   * executor's JVM is closed by a problematic task, so the JVM will notify driver that the socket
+   * is closed. If the network is connected, driver will receive the notification and trigger the
+   * function `onDisconnected`. This issue is not a `network issue` because the network is
+   * connected.
+   *
+   * [Warning 1]
+   * Worker will send heartbeats to Master every (conf.get(WORKER_TIMEOUT) * 1000 / 4) milliseconds.
+   * Check deploy/worker/Worker.scala for more details. This new mechanism design is based on the
+   * assumption: (executorTimeoutMs / 2) > (conf.get(WORKER_TIMEOUT) * 1000 / 4).
+   *
+   * [Warning 2]
+   * Not every deployment method schedules driver on master.
+   */
     logTrace("Checking for hosts with no recent heartbeats in HeartbeatReceiver.")
     val now = clock.getTimeMillis()
-    for ((executorId, lastSeenMs) <- executorLastSeen) {
-      if (now - lastSeenMs > executorTimeoutMs) {
-        logWarning(s"Removing executor $executorId with no recent heartbeats: " +
-          s"${now - lastSeenMs} ms exceeds timeout $executorTimeoutMs ms")
-        // Asynchronously kill the executor to avoid blocking the current thread
-        killExecutorThread.submit(new Runnable {
-          override def run(): Unit = Utils.tryLogNonFatalError {
-            // Note: we want to get an executor back after expiring this one,
-            // so do not simply call `sc.killExecutor` here (SPARK-8119)
-            sc.killAndReplaceExecutor(executorId)
-            // SPARK-27348: in case of the executors which are not gracefully shut down,
-            // we should remove lost executors from CoarseGrainedSchedulerBackend manually
-            // here to guarantee two things:
-            // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
-            //    a lost executor instead of waiting for disconnect message
-            // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
-            //    those executors to avoid app hang
-            sc.schedulerBackend match {
-              case backend: CoarseGrainedSchedulerBackend =>
-                backend.driverEndpoint.send(RemoveExecutor(executorId,
-                  ExecutorProcessLost(
-                    s"Executor heartbeat timed out after ${now - lastSeenMs} ms")))
-
-              // LocalSchedulerBackend is used locally and only has one single executor
-              case _: LocalSchedulerBackend =>
-
-              case other => throw new UnsupportedOperationException(
-                s"Unknown scheduler backend: ${other.getClass}")
-            }
+    if (!sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT)) {
+      for ((executorId, lastSeenMs) <- executorLastSeen) {
+        if (now - lastSeenMs > executorTimeoutMs) {
+          killExecutor(executorId, now - lastSeenMs)
+          waitingList.remove(executorId)
+          executorLastSeen.remove(executorId)
+        }
+      }
+    } else {
+      for ((executorId, workerLastHeartbeat) <- waitingList) {
+        if (now - workerLastHeartbeat > executorTimeoutMs / 2) {
+          killExecutor(executorId, now - workerLastHeartbeat)
+          waitingList.remove(executorId)
+          executorLastSeen.remove(executorId)
+        }
+      }
+
+      val buf = new ArrayBuffer[String]()
+      for ((executorId, lastSeenMs) <- executorLastSeen) {
+        if (now - lastSeenMs > executorTimeoutMs) {
+          sc.schedulerBackend match {
+            case _: StandaloneSchedulerBackend =>
+              buf += executorId
+            case _ =>
+              killExecutor(executorId, now - lastSeenMs)
+              waitingList.remove(executorId)
+              executorLastSeen.remove(executorId)
+          }
+        }
+      }
+
+      sc.schedulerBackend match {
+        case backend: StandaloneSchedulerBackend =>
+          backend.client.workerLastHeartbeat(sc.applicationId, buf) match {
+            case Some(workerLastHeartbeats) =>
+              for ((executorId, workerLastHeartbeat) <- buf zip workerLastHeartbeats) {
+                if (now - workerLastHeartbeat > executorTimeoutMs / 2) {
+                  val lastSeenMs = executorLastSeen.get(executorId).get
+                  killExecutor(executorId, now - lastSeenMs)
+                  waitingList.remove(executorId)
+                } else {
+                  waitingList(executorId) = workerLastHeartbeat
+                }
+                executorLastSeen.remove(executorId)
+              }
+            case None =>
+              for (executorId <- buf) {
+                val lastSeenMs = executorLastSeen.get(executorId).get
+                killExecutor(executorId, now - lastSeenMs)
+                executorLastSeen.remove(executorId)
+                waitingList.remove(executorId)
+              }

Review Comment:
   Discussed below.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r942063379


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -77,17 +77,44 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
 
   private[spark] var scheduler: TaskScheduler = null
 
-  // executor ID -> timestamp of when the last heartbeat from this executor was received
+  /**
+   * [SPARK-39984]
+   * Please make sure the intersection between `executorLastSeen` and `waitingList` is an empty set.
+   * If the intersection is not empty, it is possible to never kill the executor until the executor
+   * recovers. When an executor is in both `executorLastSeen` and `waitingList`, the value of
+   * `workerLastHeartbeat` in waitingList may update if the worker sends heartbeats to master
+   * normally.
+   *
+   * `executorLastSeen`:
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from this executor was received
+   *
+   *  `waitingList`: executor ID -> WorkerLastHeartbeat
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from the worker was received
+   *
+   * when driver does not receive any heartbeat from an executor for `executorTimeoutMs` seconds,
+   * the driver will ask master for the last heartbeat from the worker which the executor is running
+   * on.
+   */
   private val executorLastSeen = new HashMap[String, Long]
+  private val waitingList = new HashMap[String, Long]
 
   private val executorTimeoutMs = sc.conf.get(
     config.STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT
-  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s"))
+  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_EXECUTOR_TIMEOUT)}s"))
 
   private val checkTimeoutIntervalMs = sc.conf.get(Network.NETWORK_TIMEOUT_INTERVAL)
 
   private val executorHeartbeatIntervalMs = sc.conf.get(config.EXECUTOR_HEARTBEAT_INTERVAL)
 
+  /**
+   * Currently, [SPARK-39984] is only for StandaloneSchedulerBackend.
+   */
+  private val checkWorkerLastHeartbeat =
+    sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT) &&
+      sc.schedulerBackend.isInstanceOf[StandaloneSchedulerBackend]

Review Comment:
   I have revisited all `sc.schedulerBackend` occurrences to prevent the exceptions caused by the lack of scheduler backend.
   
   
   1. `ExecutorRegistered` / `ExecutorRemoved` / `Heartbeat` => As mentioned above, heartbeat receiver will receive all of these messages only after SchedulerBackend is initialized.
   
   2. `killExecutor` => pattern matching is exhaustive
   
   3. `expireDeadHosts` => Handled by new commit https://github.com/apache/spark/pull/37411/commits/77192300cb6990dfc60acc3227c6ae95e0223726



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r940733267


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +222,131 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms",
+                causedByApp = !sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT))))
+
+          // LocalSchedulerBackend is used locally and only has one single executor
+          case _: LocalSchedulerBackend =>
+
+          case other => throw new UnsupportedOperationException(
+            s"Unknown scheduler backend: ${other.getClass}")
+        }
+      }
+    })
+  }
+
   private def expireDeadHosts(): Unit = {
+  /**
+   * [SC-105641]
+   * Originally, the driver’s HeartbeatReceiver will expire an executor if it does not receive any
+   * heartbeat from the executor for 120 seconds. However, 120 seconds is too long, but we will face
+   * other challenges when we try to lower the timeout threshold. To elaborate, when an executor is
+   * performing full GC, it cannot send/reply any message. Next paragraphs describe the solution to
+   * detect network disconnection between driver and executor in a short time.
+   *
+   * An executor is running on a worker but in different JVMs, and a driver is running on a master
+   * but in different JVMs. Hence, the network connection between driver/executor and master/worker
+   * is the same. Because executor and worker are running on different JVMs, worker can still send
+   * heartbeat to master when executor performs GC.
+   *
+   * For new Heartbeat Receiver, if driver does not receive any heartbeat from the executor for
+   * `executorTimeoutMs` (default: 60s) seconds, HeartbeatReceiver will send a request to master to
+   * ask for the latest heartbeat from the worker which the executor runs on `workerLastHeartbeat`.
+   * HeartbeatReceiver can determine whether the heartbeat loss is caused by network issues or other
+   * issues (e.g. GC). If the heartbeat loss is not caused by network issues, the HeartbeatReceiver
+   * will put the executor into a waitingList rather than expiring it immediately.
+   *
+   * [Note]: Definition of `network issues`
+   * Here, the definition `network issues` is the issues that related to network directly. If the
+   * network is connected, the issues do not included in `network issues`. For example, an
+   * executor's JVM is closed by a problematic task, so the JVM will notify driver that the socket
+   * is closed. If the network is connected, driver will receive the notification and trigger the
+   * function `onDisconnected`. This issue is not a `network issue` because the network is
+   * connected.
+   *
+   * [Warning 1]
+   * Worker will send heartbeats to Master every (conf.get(WORKER_TIMEOUT) * 1000 / 4) milliseconds.
+   * Check deploy/worker/Worker.scala for more details. This new mechanism design is based on the
+   * assumption: (executorTimeoutMs / 2) > (conf.get(WORKER_TIMEOUT) * 1000 / 4).
+   *
+   * [Warning 2]
+   * Not every deployment method schedules driver on master.
+   */
     logTrace("Checking for hosts with no recent heartbeats in HeartbeatReceiver.")
     val now = clock.getTimeMillis()
-    for ((executorId, lastSeenMs) <- executorLastSeen) {
-      if (now - lastSeenMs > executorTimeoutMs) {
-        logWarning(s"Removing executor $executorId with no recent heartbeats: " +
-          s"${now - lastSeenMs} ms exceeds timeout $executorTimeoutMs ms")
-        // Asynchronously kill the executor to avoid blocking the current thread
-        killExecutorThread.submit(new Runnable {
-          override def run(): Unit = Utils.tryLogNonFatalError {
-            // Note: we want to get an executor back after expiring this one,
-            // so do not simply call `sc.killExecutor` here (SPARK-8119)
-            sc.killAndReplaceExecutor(executorId)
-            // SPARK-27348: in case of the executors which are not gracefully shut down,
-            // we should remove lost executors from CoarseGrainedSchedulerBackend manually
-            // here to guarantee two things:
-            // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
-            //    a lost executor instead of waiting for disconnect message
-            // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
-            //    those executors to avoid app hang
-            sc.schedulerBackend match {
-              case backend: CoarseGrainedSchedulerBackend =>
-                backend.driverEndpoint.send(RemoveExecutor(executorId,
-                  ExecutorProcessLost(
-                    s"Executor heartbeat timed out after ${now - lastSeenMs} ms")))
-
-              // LocalSchedulerBackend is used locally and only has one single executor
-              case _: LocalSchedulerBackend =>
-
-              case other => throw new UnsupportedOperationException(
-                s"Unknown scheduler backend: ${other.getClass}")
-            }
+    if (!sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT)) {

Review Comment:
   Resolved.



##########
core/src/main/scala/org/apache/spark/internal/config/Network.scala:
##########
@@ -49,7 +49,13 @@ private[spark] object Network {
     ConfigBuilder("spark.network.timeoutInterval")
       .version("1.3.2")
       .timeConf(TimeUnit.MILLISECONDS)
-      .createWithDefaultString(STORAGE_BLOCKMANAGER_TIMEOUTINTERVAL.defaultValueString)
+      .createWithDefaultString("15s")
+
+  private[spark] val NETWORK_EXECUTOR_TIMEOUT =
+    ConfigBuilder("spark.network.executorTimeout")
+      .version("1.3.0")

Review Comment:
   Resolved.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r943105027


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +242,137 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            val isStandalone = backend.isInstanceOf[StandaloneSchedulerBackend]
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms",
+                causedByApp = !checkWorkerLastHeartbeat || !isStandalone)))

Review Comment:
   I am not sure the precise definition about "executor/jvm hangs" you mention above. Thus, I made some assumptions to explain.
   
   Assumptions:
   (1) Executor cannot send/reply any message when "executor hangs", and JVM does not crash.
   => The executor loss will be detected by HeartbeatReceiver on driver
   
   (2) Executor can send/reply messages when "executor hangs", and JVM does not crash.
   => Cannot be detected.
   
   (3) JVM crashes
   => The event will be detected by driver.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r943127905


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -130,8 +170,11 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     case heartbeat @ Heartbeat(executorId, accumUpdates, blockManagerId, executorUpdates) =>
       var reregisterBlockManager = !sc.isStopped
       if (scheduler != null) {
-        if (executorLastSeen.contains(executorId)) {
+        val isStandalone = sc.schedulerBackend.isInstanceOf[StandaloneSchedulerBackend]

Review Comment:
   Updated. https://github.com/apache/spark/pull/37411/commits/41c6bc5e062a38a068f506deacc5ce2db59a356e



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r954098890


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -77,17 +77,61 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
 
   private[spark] var scheduler: TaskScheduler = null
 
-  // executor ID -> timestamp of when the last heartbeat from this executor was received
+  /**
+   * [SPARK-39984]
+   * Please make sure the intersection between `executorLastSeen` and `executorExpiryCandidates` is
+   * an empty set. If the intersection is not empty, it is possible to never kill the executor until
+   * the executor recovers. When an executor is in both `executorLastSeen` and
+   * `executorExpiryCandidates`, the value of `workerLastHeartbeat` in `executorExpiryCandidates`
+   * may update if the worker sends heartbeats to master normally.
+   *
+   * `executorLastSeen`:
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from this executor was received
+   *
+   * `executorExpiryCandidates`: executor ID -> WorkerLastHeartbeat
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from the worker was received
+   *
+   * when driver does not receive any heartbeat from an executor for `executorTimeoutMs` seconds,
+   * the driver will ask master for the last heartbeat from the worker which the executor is running
+   * on.
+   */
   private val executorLastSeen = new HashMap[String, Long]
+  private val executorExpiryCandidates = new HashMap[String, Long]
 
   private val executorTimeoutMs = sc.conf.get(
     config.STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT
-  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s"))
+  ).getOrElse(
+    sc.conf.get(Network.NETWORK_EXECUTOR_TIMEOUT) match {
+      case Some(executorTimeout) => executorTimeout
+      case None => Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s")
+    }
+  )
 
   private val checkTimeoutIntervalMs = sc.conf.get(Network.NETWORK_TIMEOUT_INTERVAL)
 
   private val executorHeartbeatIntervalMs = sc.conf.get(config.EXECUTOR_HEARTBEAT_INTERVAL)
 
+  /**
+   * Currently, [SPARK-39984] is only for StandaloneSchedulerBackend.
+   *
+   * `checkWorkerLastHeartbeat`: A flag to enable two-phase executor timeout.
+   * `expiryCandidatesTimeout`: The timeout used for executorExpiryCandidates.
+   */
+  private val checkWorkerLastHeartbeat = {
+    val isEnabled = sc.conf.get(config.HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT)
+    if (isEnabled) logWarning(s"Keep `expiryCandidatesTimeout` larger than `HEARTBEAT_MILLIS` in" +
+      s"deploy/worker/Worker.scala to know whether master lost any heartbeat from the" +
+      s"worker or not.")
+    isEnabled
+  }
+
+  private val expiryCandidatesTimeout = checkWorkerLastHeartbeat match {
+    case true => sc.conf.get(config.HEARTBEAT_EXPIRY_CANDIDATES_TIMEOUT)

Review Comment:
   Updated https://github.com/apache/spark/pull/37411/commits/2875fb158f79ba9c9abf0aaef7dca7fe0c00839a



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] mridulm commented on pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
mridulm commented on PR #37411:
URL: https://github.com/apache/spark/pull/37411#issuecomment-1225258171

   I am fine with the changes, will let @Ngone51 take a look/merge


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r945376133


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +241,120 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            // TODO (SPARK-39984): Update causedByApp when we have a hanging task detector
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms")))
+          // LocalSchedulerBackend is used locally and only has one single executor
+          case _: LocalSchedulerBackend =>
+
+          case other => throw new UnsupportedOperationException(
+            s"Unknown scheduler backend: ${other.getClass}")
+        }
+      }
+    })
+  }
+
+  private def isStandalone(): Boolean = {
+    sc.schedulerBackend match {
+      case backend: StandaloneSchedulerBackend => true
+      case _ => false
+    }
+  }
+
+  private def removeExecutorFromExpiryCandidates(executorId: String): Unit = {
+    if (checkWorkerLastHeartbeat && isStandalone()) {
+      executorExpiryCandidates.remove(executorId)
+    }
+  }
+
   private def expireDeadHosts(): Unit = {
+  /**
+   * [SPARK-39984]
+   * The driver’s HeartbeatReceiver will expire an executor if it does not receive any heartbeat
+   * from the executor for `executorTimeoutMs` (default 120s) seconds. However, lowering from 120
+   * seconds has other challenges. For example: when executor is performing full GC, it cannot
+   * send/reply any message for tens of seconds (based on your environment). Hence,
+   * HeartbeatReceiver cannot whether the heartbeat loss is caused by network issues or other
+   * reasons (e.g. full GC). To address this, we designed a new Heartbeat Receiver mechanism for
+   * standalone deployments.
+   *
+   * For standalone deployments:
+   * If driver does not receive any heartbeat from the executor for `executorTimeoutMs` seconds,
+   * HeartbeatReceiver will send a request to master to ask for the latest heartbeat from the
+   * worker which the executor runs on. HeartbeatReceiver can determine whether the heartbeat loss
+   * is caused by network issues or other issues (e.g. GC). If the heartbeat loss is not caused by
+   * network issues, the HeartbeatReceiver will put the executor into `executorExpiryCandidates`
+   * rather than expiring it immediately.
+   */
     logTrace("Checking for hosts with no recent heartbeats in HeartbeatReceiver.")
+    logWarning(s"Keep `expiryCandidatesTimeout` larger than `HEARTBEAT_MILLIS` in" +
+      s"deploy/worker/Worker.scala to know whether master lost any heartbeat from the" +
+      s"worker or not.")
     val now = clock.getTimeMillis()
-    for ((executorId, lastSeenMs) <- executorLastSeen) {
-      if (now - lastSeenMs > executorTimeoutMs) {
-        logWarning(s"Removing executor $executorId with no recent heartbeats: " +
-          s"${now - lastSeenMs} ms exceeds timeout $executorTimeoutMs ms")
-        // Asynchronously kill the executor to avoid blocking the current thread
-        killExecutorThread.submit(new Runnable {
-          override def run(): Unit = Utils.tryLogNonFatalError {
-            // Note: we want to get an executor back after expiring this one,
-            // so do not simply call `sc.killExecutor` here (SPARK-8119)
-            sc.killAndReplaceExecutor(executorId)
-            // SPARK-27348: in case of the executors which are not gracefully shut down,
-            // we should remove lost executors from CoarseGrainedSchedulerBackend manually
-            // here to guarantee two things:
-            // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
-            //    a lost executor instead of waiting for disconnect message
-            // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
-            //    those executors to avoid app hang
-            sc.schedulerBackend match {
-              case backend: CoarseGrainedSchedulerBackend =>
-                backend.driverEndpoint.send(RemoveExecutor(executorId,
-                  ExecutorProcessLost(
-                    s"Executor heartbeat timed out after ${now - lastSeenMs} ms")))
-
-              // LocalSchedulerBackend is used locally and only has one single executor
-              case _: LocalSchedulerBackend =>
-
-              case other => throw new UnsupportedOperationException(
-                s"Unknown scheduler backend: ${other.getClass}")
-            }
+    if (!checkWorkerLastHeartbeat || !isStandalone()) {
+      for ((executorId, lastSeenMs) <- executorLastSeen) {
+        if (now - lastSeenMs > executorTimeoutMs) {
+          killExecutor(executorId, now - lastSeenMs)
+          executorLastSeen.remove(executorId)
+        }
+      }
+    } else {
+      for ((executorId, workerLastHeartbeat) <- executorExpiryCandidates) {
+        if (now - workerLastHeartbeat > expiryCandidatesTimeout) {
+          killExecutor(executorId, now - workerLastHeartbeat)
+          executorExpiryCandidates.remove(executorId)
+          executorLastSeen.remove(executorId)
+        }
+      }
+
+      val buf = new ArrayBuffer[String]()
+      for ((executorId, lastSeenMs) <- executorLastSeen) {
+        if (now - lastSeenMs > executorTimeoutMs) {
+            buf += executorId
+        }
+      }
+
+      sc.schedulerBackend match {
+        case backend: StandaloneSchedulerBackend =>
+          backend.client.workerLastHeartbeat(sc.applicationId, buf) match {
+            case Some(workerLastHeartbeats) =>
+              for ((executorId, workerLastHeartbeat) <- buf zip workerLastHeartbeats) {
+                if (now - workerLastHeartbeat > expiryCandidatesTimeout) {
+                  val lastSeenMs = executorLastSeen.get(executorId).get

Review Comment:
   ```suggestion
                     val lastSeenMs = executorLastSeen(executorId)
   ```



##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +241,120 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            // TODO (SPARK-39984): Update causedByApp when we have a hanging task detector
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms")))
+          // LocalSchedulerBackend is used locally and only has one single executor
+          case _: LocalSchedulerBackend =>
+
+          case other => throw new UnsupportedOperationException(
+            s"Unknown scheduler backend: ${other.getClass}")
+        }
+      }
+    })
+  }
+
+  private def isStandalone(): Boolean = {
+    sc.schedulerBackend match {
+      case backend: StandaloneSchedulerBackend => true
+      case _ => false
+    }

Review Comment:
   ```suggestion
       sc.schedulerBackend.isInstanceOf[StandaloneSchedulerBackend]
     }
   ```



##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +241,120 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            // TODO (SPARK-39984): Update causedByApp when we have a hanging task detector
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms")))
+          // LocalSchedulerBackend is used locally and only has one single executor
+          case _: LocalSchedulerBackend =>
+
+          case other => throw new UnsupportedOperationException(
+            s"Unknown scheduler backend: ${other.getClass}")
+        }
+      }
+    })
+  }
+
+  private def isStandalone(): Boolean = {
+    sc.schedulerBackend match {
+      case backend: StandaloneSchedulerBackend => true
+      case _ => false
+    }
+  }
+
+  private def removeExecutorFromExpiryCandidates(executorId: String): Unit = {
+    if (checkWorkerLastHeartbeat && isStandalone()) {
+      executorExpiryCandidates.remove(executorId)
+    }
+  }
+
   private def expireDeadHosts(): Unit = {
+  /**
+   * [SPARK-39984]
+   * The driver’s HeartbeatReceiver will expire an executor if it does not receive any heartbeat
+   * from the executor for `executorTimeoutMs` (default 120s) seconds. However, lowering from 120
+   * seconds has other challenges. For example: when executor is performing full GC, it cannot
+   * send/reply any message for tens of seconds (based on your environment). Hence,
+   * HeartbeatReceiver cannot whether the heartbeat loss is caused by network issues or other
+   * reasons (e.g. full GC). To address this, we designed a new Heartbeat Receiver mechanism for
+   * standalone deployments.
+   *
+   * For standalone deployments:
+   * If driver does not receive any heartbeat from the executor for `executorTimeoutMs` seconds,
+   * HeartbeatReceiver will send a request to master to ask for the latest heartbeat from the
+   * worker which the executor runs on. HeartbeatReceiver can determine whether the heartbeat loss
+   * is caused by network issues or other issues (e.g. GC). If the heartbeat loss is not caused by
+   * network issues, the HeartbeatReceiver will put the executor into `executorExpiryCandidates`
+   * rather than expiring it immediately.
+   */
     logTrace("Checking for hosts with no recent heartbeats in HeartbeatReceiver.")
+    logWarning(s"Keep `expiryCandidatesTimeout` larger than `HEARTBEAT_MILLIS` in" +

Review Comment:
   We should only log this once. For example, we could log this during `checkWorkerLastHeartbeat` initialization:
   
   ```
   private val checkWorkerLastHeartbeat = {
     val isEnabled = sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT)
     if (isEnabled) logWarning(...)
     isEnabled
   }
   
   ```



##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +241,120 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            // TODO (SPARK-39984): Update causedByApp when we have a hanging task detector
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms")))
+          // LocalSchedulerBackend is used locally and only has one single executor
+          case _: LocalSchedulerBackend =>
+
+          case other => throw new UnsupportedOperationException(
+            s"Unknown scheduler backend: ${other.getClass}")
+        }
+      }
+    })
+  }
+
+  private def isStandalone(): Boolean = {
+    sc.schedulerBackend match {
+      case backend: StandaloneSchedulerBackend => true
+      case _ => false
+    }
+  }
+
+  private def removeExecutorFromExpiryCandidates(executorId: String): Unit = {
+    if (checkWorkerLastHeartbeat && isStandalone()) {
+      executorExpiryCandidates.remove(executorId)
+    }
+  }
+
   private def expireDeadHosts(): Unit = {
+  /**
+   * [SPARK-39984]
+   * The driver’s HeartbeatReceiver will expire an executor if it does not receive any heartbeat
+   * from the executor for `executorTimeoutMs` (default 120s) seconds. However, lowering from 120
+   * seconds has other challenges. For example: when executor is performing full GC, it cannot
+   * send/reply any message for tens of seconds (based on your environment). Hence,
+   * HeartbeatReceiver cannot whether the heartbeat loss is caused by network issues or other
+   * reasons (e.g. full GC). To address this, we designed a new Heartbeat Receiver mechanism for
+   * standalone deployments.
+   *
+   * For standalone deployments:
+   * If driver does not receive any heartbeat from the executor for `executorTimeoutMs` seconds,
+   * HeartbeatReceiver will send a request to master to ask for the latest heartbeat from the
+   * worker which the executor runs on. HeartbeatReceiver can determine whether the heartbeat loss
+   * is caused by network issues or other issues (e.g. GC). If the heartbeat loss is not caused by
+   * network issues, the HeartbeatReceiver will put the executor into `executorExpiryCandidates`
+   * rather than expiring it immediately.
+   */
     logTrace("Checking for hosts with no recent heartbeats in HeartbeatReceiver.")
+    logWarning(s"Keep `expiryCandidatesTimeout` larger than `HEARTBEAT_MILLIS` in" +
+      s"deploy/worker/Worker.scala to know whether master lost any heartbeat from the" +
+      s"worker or not.")
     val now = clock.getTimeMillis()
-    for ((executorId, lastSeenMs) <- executorLastSeen) {
-      if (now - lastSeenMs > executorTimeoutMs) {
-        logWarning(s"Removing executor $executorId with no recent heartbeats: " +
-          s"${now - lastSeenMs} ms exceeds timeout $executorTimeoutMs ms")
-        // Asynchronously kill the executor to avoid blocking the current thread
-        killExecutorThread.submit(new Runnable {
-          override def run(): Unit = Utils.tryLogNonFatalError {
-            // Note: we want to get an executor back after expiring this one,
-            // so do not simply call `sc.killExecutor` here (SPARK-8119)
-            sc.killAndReplaceExecutor(executorId)
-            // SPARK-27348: in case of the executors which are not gracefully shut down,
-            // we should remove lost executors from CoarseGrainedSchedulerBackend manually
-            // here to guarantee two things:
-            // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
-            //    a lost executor instead of waiting for disconnect message
-            // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
-            //    those executors to avoid app hang
-            sc.schedulerBackend match {
-              case backend: CoarseGrainedSchedulerBackend =>
-                backend.driverEndpoint.send(RemoveExecutor(executorId,
-                  ExecutorProcessLost(
-                    s"Executor heartbeat timed out after ${now - lastSeenMs} ms")))
-
-              // LocalSchedulerBackend is used locally and only has one single executor
-              case _: LocalSchedulerBackend =>
-
-              case other => throw new UnsupportedOperationException(
-                s"Unknown scheduler backend: ${other.getClass}")
-            }
+    if (!checkWorkerLastHeartbeat || !isStandalone()) {
+      for ((executorId, lastSeenMs) <- executorLastSeen) {
+        if (now - lastSeenMs > executorTimeoutMs) {
+          killExecutor(executorId, now - lastSeenMs)
+          executorLastSeen.remove(executorId)
+        }
+      }
+    } else {
+      for ((executorId, workerLastHeartbeat) <- executorExpiryCandidates) {
+        if (now - workerLastHeartbeat > expiryCandidatesTimeout) {
+          killExecutor(executorId, now - workerLastHeartbeat)
+          executorExpiryCandidates.remove(executorId)
+          executorLastSeen.remove(executorId)
+        }
+      }
+
+      val buf = new ArrayBuffer[String]()
+      for ((executorId, lastSeenMs) <- executorLastSeen) {
+        if (now - lastSeenMs > executorTimeoutMs) {
+            buf += executorId
+        }
+      }
+
+      sc.schedulerBackend match {
+        case backend: StandaloneSchedulerBackend =>
+          backend.client.workerLastHeartbeat(sc.applicationId, buf) match {
+            case Some(workerLastHeartbeats) =>
+              for ((executorId, workerLastHeartbeat) <- buf zip workerLastHeartbeats) {
+                if (now - workerLastHeartbeat > expiryCandidatesTimeout) {
+                  val lastSeenMs = executorLastSeen.get(executorId).get
+                  killExecutor(executorId, now - lastSeenMs)
+                  executorExpiryCandidates.remove(executorId)
+                } else {
+                  executorExpiryCandidates(executorId) = workerLastHeartbeat
+                }
+                executorLastSeen.remove(executorId)
+              }
+            case None =>
+              for (executorId <- buf) {
+                val lastSeenMs = executorLastSeen.get(executorId).get

Review Comment:
   ```suggestion
                   val lastSeenMs = executorLastSeen(executorId)
   ```



##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +241,120 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            // TODO (SPARK-39984): Update causedByApp when we have a hanging task detector
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms")))
+          // LocalSchedulerBackend is used locally and only has one single executor
+          case _: LocalSchedulerBackend =>
+
+          case other => throw new UnsupportedOperationException(
+            s"Unknown scheduler backend: ${other.getClass}")
+        }
+      }
+    })
+  }
+
+  private def isStandalone(): Boolean = {
+    sc.schedulerBackend match {
+      case backend: StandaloneSchedulerBackend => true
+      case _ => false
+    }
+  }
+
+  private def removeExecutorFromExpiryCandidates(executorId: String): Unit = {
+    if (checkWorkerLastHeartbeat && isStandalone()) {
+      executorExpiryCandidates.remove(executorId)
+    }
+  }
+
   private def expireDeadHosts(): Unit = {
+  /**
+   * [SPARK-39984]
+   * The driver’s HeartbeatReceiver will expire an executor if it does not receive any heartbeat
+   * from the executor for `executorTimeoutMs` (default 120s) seconds. However, lowering from 120
+   * seconds has other challenges. For example: when executor is performing full GC, it cannot
+   * send/reply any message for tens of seconds (based on your environment). Hence,
+   * HeartbeatReceiver cannot whether the heartbeat loss is caused by network issues or other
+   * reasons (e.g. full GC). To address this, we designed a new Heartbeat Receiver mechanism for
+   * standalone deployments.
+   *
+   * For standalone deployments:
+   * If driver does not receive any heartbeat from the executor for `executorTimeoutMs` seconds,
+   * HeartbeatReceiver will send a request to master to ask for the latest heartbeat from the
+   * worker which the executor runs on. HeartbeatReceiver can determine whether the heartbeat loss
+   * is caused by network issues or other issues (e.g. GC). If the heartbeat loss is not caused by
+   * network issues, the HeartbeatReceiver will put the executor into `executorExpiryCandidates`
+   * rather than expiring it immediately.
+   */
     logTrace("Checking for hosts with no recent heartbeats in HeartbeatReceiver.")
+    logWarning(s"Keep `expiryCandidatesTimeout` larger than `HEARTBEAT_MILLIS` in" +
+      s"deploy/worker/Worker.scala to know whether master lost any heartbeat from the" +
+      s"worker or not.")
     val now = clock.getTimeMillis()
-    for ((executorId, lastSeenMs) <- executorLastSeen) {
-      if (now - lastSeenMs > executorTimeoutMs) {
-        logWarning(s"Removing executor $executorId with no recent heartbeats: " +
-          s"${now - lastSeenMs} ms exceeds timeout $executorTimeoutMs ms")
-        // Asynchronously kill the executor to avoid blocking the current thread
-        killExecutorThread.submit(new Runnable {
-          override def run(): Unit = Utils.tryLogNonFatalError {
-            // Note: we want to get an executor back after expiring this one,
-            // so do not simply call `sc.killExecutor` here (SPARK-8119)
-            sc.killAndReplaceExecutor(executorId)
-            // SPARK-27348: in case of the executors which are not gracefully shut down,
-            // we should remove lost executors from CoarseGrainedSchedulerBackend manually
-            // here to guarantee two things:
-            // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
-            //    a lost executor instead of waiting for disconnect message
-            // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
-            //    those executors to avoid app hang
-            sc.schedulerBackend match {
-              case backend: CoarseGrainedSchedulerBackend =>
-                backend.driverEndpoint.send(RemoveExecutor(executorId,
-                  ExecutorProcessLost(
-                    s"Executor heartbeat timed out after ${now - lastSeenMs} ms")))
-
-              // LocalSchedulerBackend is used locally and only has one single executor
-              case _: LocalSchedulerBackend =>
-
-              case other => throw new UnsupportedOperationException(
-                s"Unknown scheduler backend: ${other.getClass}")
-            }
+    if (!checkWorkerLastHeartbeat || !isStandalone()) {
+      for ((executorId, lastSeenMs) <- executorLastSeen) {
+        if (now - lastSeenMs > executorTimeoutMs) {
+          killExecutor(executorId, now - lastSeenMs)
+          executorLastSeen.remove(executorId)
+        }
+      }
+    } else {
+      for ((executorId, workerLastHeartbeat) <- executorExpiryCandidates) {
+        if (now - workerLastHeartbeat > expiryCandidatesTimeout) {
+          killExecutor(executorId, now - workerLastHeartbeat)
+          executorExpiryCandidates.remove(executorId)
+          executorLastSeen.remove(executorId)
+        }
+      }

Review Comment:
   Could we move this block to the end of this `else` branch? And then we don't need to kill executors while handling `workerLastHeartbeats` below but do all kills in the same block. E.g.,
   
   ```scala
   val buf = new ArrayBuffer[String]()
   for ((executorId, lastSeenMs) <- executorLastSeen) {
     if (now - lastSeenMs > executorTimeoutMs) {
         buf += executorId
     }
   }
   sc.schedulerBackend match {
     case backend: StandaloneSchedulerBackend =>
       backend.client.workerLastHeartbeat(sc.applicationId, buf) match {
         case Some(workerLastHeartbeats) =>
           for ((executorId, workerLastHeartbeat) <- buf zip workerLastHeartbeats) {
              executorExpiryCandidates(executorId) = workerLastHeartbeat
              executorLastSeen.remove(executorId)
           }
          case None =>
            for (executorId <- buf) {
              val lastSeenMs = executorLastSeen.get(executorId).get
              executorExpiryCandidates(executoId) =Long.Max
            }
   
   for ((executorId, workerLastHeartbeat) <- executorExpiryCandidates) {
     if (now - workerLastHeartbeat > expiryCandidatesTimeout) {
      // One problem here might be that  we can not use `workerLastHeartbeat` for the None case as it was 
      // lastSeenMs previously. 
       killExecutor(executorId, now - workerLastHeartbeat)
       executorExpiryCandidates.remove(executorId)
       executorLastSeen.remove(executorId)
     }
   }
   ```
   



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r946378989


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +241,120 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            // TODO (SPARK-39984): Update causedByApp when we have a hanging task detector
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms")))
+          // LocalSchedulerBackend is used locally and only has one single executor
+          case _: LocalSchedulerBackend =>
+
+          case other => throw new UnsupportedOperationException(
+            s"Unknown scheduler backend: ${other.getClass}")
+        }
+      }
+    })
+  }
+
+  private def isStandalone(): Boolean = {
+    sc.schedulerBackend match {
+      case backend: StandaloneSchedulerBackend => true
+      case _ => false
+    }

Review Comment:
   Applied the suggestions.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r945367796


##########
core/src/main/scala/org/apache/spark/internal/config/Network.scala:
##########
@@ -51,6 +51,21 @@ private[spark] object Network {
       .timeConf(TimeUnit.MILLISECONDS)
       .createWithDefaultString(STORAGE_BLOCKMANAGER_TIMEOUTINTERVAL.defaultValueString)
 
+  private[spark] val NETWORK_EXECUTOR_TIMEOUT =
+    ConfigBuilder("spark.network.executorTimeout")
+      .version("3.4.0")
+      .timeConf(TimeUnit.MILLISECONDS)
+      .createOptional
+
+  private[spark] val HEARTBEAT_EXPIRY_CANDIDATES_TIMEOUT =
+    ConfigBuilder("spark.network.expiryCandidatesTimeout")
+      .doc("This config is a timeout used for heartbeat receiver `executorExpiryCandidates`. Be" +
+        "effective only when HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT is enabled. See" +
+        "[SPARK-39984] for more details")
+      .version("3.4.0")
+      .timeConf(TimeUnit.MILLISECONDS)
+      .createWithDefaultString("30s")

Review Comment:
   +1. 
   
   We can add this config to the namespace `spark.driver.heartbeat`. cc @kevin85421 



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r954138822


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -77,17 +77,61 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
 
   private[spark] var scheduler: TaskScheduler = null
 
-  // executor ID -> timestamp of when the last heartbeat from this executor was received
+  /**
+   * [SPARK-39984]
+   * Please make sure the intersection between `executorLastSeen` and `executorExpiryCandidates` is
+   * an empty set. If the intersection is not empty, it is possible to never kill the executor until
+   * the executor recovers. When an executor is in both `executorLastSeen` and
+   * `executorExpiryCandidates`, the value of `workerLastHeartbeat` in `executorExpiryCandidates`
+   * may update if the worker sends heartbeats to master normally.
+   *
+   * `executorLastSeen`:
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from this executor was received
+   *
+   * `executorExpiryCandidates`: executor ID -> WorkerLastHeartbeat
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from the worker was received
+   *
+   * when driver does not receive any heartbeat from an executor for `executorTimeoutMs` seconds,
+   * the driver will ask master for the last heartbeat from the worker which the executor is running
+   * on.
+   */
   private val executorLastSeen = new HashMap[String, Long]
+  private val executorExpiryCandidates = new HashMap[String, Long]
 
   private val executorTimeoutMs = sc.conf.get(
     config.STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT
-  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s"))
+  ).getOrElse(
+    sc.conf.get(Network.NETWORK_EXECUTOR_TIMEOUT) match {
+      case Some(executorTimeout) => executorTimeout
+      case None => Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s")
+    }
+  )
 
   private val checkTimeoutIntervalMs = sc.conf.get(Network.NETWORK_TIMEOUT_INTERVAL)
 
   private val executorHeartbeatIntervalMs = sc.conf.get(config.EXECUTOR_HEARTBEAT_INTERVAL)
 
+  /**
+   * Currently, [SPARK-39984] is only for StandaloneSchedulerBackend.
+   *
+   * `checkWorkerLastHeartbeat`: A flag to enable two-phase executor timeout.
+   * `expiryCandidatesTimeout`: The timeout used for executorExpiryCandidates.
+   */
+  private val checkWorkerLastHeartbeat = {

Review Comment:
   This is a good point, but there is a question. 
   
   In my understanding, your comment is as following code. 
   ```scala
     private lazy val checkWorkerLastHeartbeat =
       sc.conf.get(config.HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT)
         && sc.schedulerBackend.isInstanceOf[StandaloneSchedulerBackend]
   ```
   In the function `expireDeadHosts`, we will check the value of `checkWorkerLastHeartbeat`, so this variable will be evaluated. However, if the first `expireDeadHosts` trigger is prior to scheduler backend initialization, the value of `checkWorkerLastHeartbeat` will be false (`sc.schedulerBackend.isInstanceOf[StandaloneSchedulerBackend]` == false). If the scheduler backend initialization finishes before the second `expireDeadHosts` trigger, the value of `checkWorkerLastHeartbeat` should be true. Nevertheless, it is set to false in the first `expireDeadHosts` function call.
   
   
   Maybe the other solution is to define `checkWorkerLastHeartbeat` as a function rather than a variable. Any thoughts?



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r953765470


##########
core/src/main/scala/org/apache/spark/internal/config/package.scala:
##########
@@ -2398,4 +2398,20 @@ package object config {
       .version("3.3.0")
       .intConf
       .createWithDefault(5)
+
+  private[spark] val HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT =
+    ConfigBuilder("spark.driver.heartbeat.checkWorkerLastHeartbeat")
+      .internal()

Review Comment:
   Add doc for it and mention that this only works for Standalone.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r953806461


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -77,17 +77,61 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
 
   private[spark] var scheduler: TaskScheduler = null
 
-  // executor ID -> timestamp of when the last heartbeat from this executor was received
+  /**
+   * [SPARK-39984]
+   * Please make sure the intersection between `executorLastSeen` and `executorExpiryCandidates` is
+   * an empty set. If the intersection is not empty, it is possible to never kill the executor until
+   * the executor recovers. When an executor is in both `executorLastSeen` and
+   * `executorExpiryCandidates`, the value of `workerLastHeartbeat` in `executorExpiryCandidates`
+   * may update if the worker sends heartbeats to master normally.
+   *
+   * `executorLastSeen`:
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from this executor was received
+   *
+   * `executorExpiryCandidates`: executor ID -> WorkerLastHeartbeat
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from the worker was received
+   *
+   * when driver does not receive any heartbeat from an executor for `executorTimeoutMs` seconds,
+   * the driver will ask master for the last heartbeat from the worker which the executor is running
+   * on.
+   */
   private val executorLastSeen = new HashMap[String, Long]
+  private val executorExpiryCandidates = new HashMap[String, Long]
 
   private val executorTimeoutMs = sc.conf.get(
     config.STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT
-  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s"))
+  ).getOrElse(
+    sc.conf.get(Network.NETWORK_EXECUTOR_TIMEOUT) match {
+      case Some(executorTimeout) => executorTimeout
+      case None => Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s")
+    }
+  )
 
   private val checkTimeoutIntervalMs = sc.conf.get(Network.NETWORK_TIMEOUT_INTERVAL)
 
   private val executorHeartbeatIntervalMs = sc.conf.get(config.EXECUTOR_HEARTBEAT_INTERVAL)
 
+  /**
+   * Currently, [SPARK-39984] is only for StandaloneSchedulerBackend.
+   *
+   * `checkWorkerLastHeartbeat`: A flag to enable two-phase executor timeout.
+   * `expiryCandidatesTimeout`: The timeout used for executorExpiryCandidates.
+   */
+  private val checkWorkerLastHeartbeat = {
+    val isEnabled = sc.conf.get(config.HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT)
+    if (isEnabled) logWarning(s"Keep `expiryCandidatesTimeout` larger than `HEARTBEAT_MILLIS` in" +
+      s"deploy/worker/Worker.scala to know whether master lost any heartbeat from the" +
+      s"worker or not.")
+    isEnabled
+  }
+
+  private val expiryCandidatesTimeout = checkWorkerLastHeartbeat match {

Review Comment:
   lazy too



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r940935657


##########
core/src/main/scala/org/apache/spark/internal/config/Network.scala:
##########
@@ -49,7 +49,13 @@ private[spark] object Network {
     ConfigBuilder("spark.network.timeoutInterval")
       .version("1.3.2")
       .timeConf(TimeUnit.MILLISECONDS)
-      .createWithDefaultString(STORAGE_BLOCKMANAGER_TIMEOUTINTERVAL.defaultValueString)
+      .createWithDefaultString("15s")
+
+  private[spark] val NETWORK_EXECUTOR_TIMEOUT =
+    ConfigBuilder("spark.network.executorTimeout")
+      .version("1.3.0")
+      .timeConf(TimeUnit.MILLISECONDS)
+      .createWithDefaultString("60s")

Review Comment:
   Updated. I change `NETWORK_EXECUTOR_TIMEOUT` to `Option`. In addition, I write two tests to test this change.
   
   ```scala
   private val executorTimeoutMs = sc.conf.get(
     config.STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT
   ).getOrElse(
     sc.conf.get(Network.NETWORK_EXECUTOR_TIMEOUT) match {
       case Some(executorTimeout) => Utils.timeStringAsMs(s"${executorTimeout}s")
       case None => Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s")
     }
   )
   ```
   
   * Test
   ```bash
   build/sbt "core/testOnly *HeartbeatReceiverSuite -- -z NETWORK_EXECUTOR_TIMEOUT"
   ``` 



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] mridulm commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r942900759


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -77,17 +77,44 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
 
   private[spark] var scheduler: TaskScheduler = null
 
-  // executor ID -> timestamp of when the last heartbeat from this executor was received
+  /**
+   * [SPARK-39984]
+   * Please make sure the intersection between `executorLastSeen` and `waitingList` is an empty set.
+   * If the intersection is not empty, it is possible to never kill the executor until the executor
+   * recovers. When an executor is in both `executorLastSeen` and `waitingList`, the value of
+   * `workerLastHeartbeat` in waitingList may update if the worker sends heartbeats to master
+   * normally.
+   *
+   * `executorLastSeen`:
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from this executor was received
+   *
+   *  `waitingList`: executor ID -> WorkerLastHeartbeat
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from the worker was received
+   *
+   * when driver does not receive any heartbeat from an executor for `executorTimeoutMs` seconds,
+   * the driver will ask master for the last heartbeat from the worker which the executor is running
+   * on.
+   */
   private val executorLastSeen = new HashMap[String, Long]
+  private val waitingList = new HashMap[String, Long]
 
   private val executorTimeoutMs = sc.conf.get(
     config.STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT
-  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s"))
+  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_EXECUTOR_TIMEOUT)}s"))
 
   private val checkTimeoutIntervalMs = sc.conf.get(Network.NETWORK_TIMEOUT_INTERVAL)
 
   private val executorHeartbeatIntervalMs = sc.conf.get(config.EXECUTOR_HEARTBEAT_INTERVAL)
 
+  /**
+   * Currently, [SPARK-39984] is only for StandaloneSchedulerBackend.
+   */
+  private val checkWorkerLastHeartbeat =
+    sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT) &&
+      sc.schedulerBackend.isInstanceOf[StandaloneSchedulerBackend]

Review Comment:
   Note that you can actually have `ExecutorRegistered` before `sc.schedulerBackend` is set - this is essentially a race condition. This is why we have a `if (scheduler != null) {` check while handling `Heartbeat`



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] mridulm commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r942900759


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -77,17 +77,44 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
 
   private[spark] var scheduler: TaskScheduler = null
 
-  // executor ID -> timestamp of when the last heartbeat from this executor was received
+  /**
+   * [SPARK-39984]
+   * Please make sure the intersection between `executorLastSeen` and `waitingList` is an empty set.
+   * If the intersection is not empty, it is possible to never kill the executor until the executor
+   * recovers. When an executor is in both `executorLastSeen` and `waitingList`, the value of
+   * `workerLastHeartbeat` in waitingList may update if the worker sends heartbeats to master
+   * normally.
+   *
+   * `executorLastSeen`:
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from this executor was received
+   *
+   *  `waitingList`: executor ID -> WorkerLastHeartbeat
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from the worker was received
+   *
+   * when driver does not receive any heartbeat from an executor for `executorTimeoutMs` seconds,
+   * the driver will ask master for the last heartbeat from the worker which the executor is running
+   * on.
+   */
   private val executorLastSeen = new HashMap[String, Long]
+  private val waitingList = new HashMap[String, Long]
 
   private val executorTimeoutMs = sc.conf.get(
     config.STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT
-  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s"))
+  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_EXECUTOR_TIMEOUT)}s"))
 
   private val checkTimeoutIntervalMs = sc.conf.get(Network.NETWORK_TIMEOUT_INTERVAL)
 
   private val executorHeartbeatIntervalMs = sc.conf.get(config.EXECUTOR_HEARTBEAT_INTERVAL)
 
+  /**
+   * Currently, [SPARK-39984] is only for StandaloneSchedulerBackend.
+   */
+  private val checkWorkerLastHeartbeat =
+    sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT) &&
+      sc.schedulerBackend.isInstanceOf[StandaloneSchedulerBackend]

Review Comment:
   There are some corner cases here you might want to check up on.
   For example, we have a `if (scheduler != null) {` check while handling `Heartbeat`.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] mridulm commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r942958914


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -77,17 +77,44 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
 
   private[spark] var scheduler: TaskScheduler = null
 
-  // executor ID -> timestamp of when the last heartbeat from this executor was received
+  /**
+   * [SPARK-39984]
+   * Please make sure the intersection between `executorLastSeen` and `waitingList` is an empty set.
+   * If the intersection is not empty, it is possible to never kill the executor until the executor
+   * recovers. When an executor is in both `executorLastSeen` and `waitingList`, the value of
+   * `workerLastHeartbeat` in waitingList may update if the worker sends heartbeats to master
+   * normally.
+   *
+   * `executorLastSeen`:
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from this executor was received
+   *
+   *  `waitingList`: executor ID -> WorkerLastHeartbeat
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from the worker was received
+   *
+   * when driver does not receive any heartbeat from an executor for `executorTimeoutMs` seconds,
+   * the driver will ask master for the last heartbeat from the worker which the executor is running
+   * on.
+   */
   private val executorLastSeen = new HashMap[String, Long]
+  private val waitingList = new HashMap[String, Long]
 
   private val executorTimeoutMs = sc.conf.get(
     config.STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT
-  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s"))
+  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_EXECUTOR_TIMEOUT)}s"))
 
   private val checkTimeoutIntervalMs = sc.conf.get(Network.NETWORK_TIMEOUT_INTERVAL)
 
   private val executorHeartbeatIntervalMs = sc.conf.get(config.EXECUTOR_HEARTBEAT_INTERVAL)
 
+  /**
+   * Currently, [SPARK-39984] is only for StandaloneSchedulerBackend.
+   */
+  private val checkWorkerLastHeartbeat =
+    sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT) &&
+      sc.schedulerBackend.isInstanceOf[StandaloneSchedulerBackend]

Review Comment:
   This code has evolved a lot, so it is possible that the check in handling `Heartbeat` is no longer valid (the `null` check there is equally not likely given your comment above) - having said that, I have not looked at all corner cases, so would be careful.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r942969528


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -77,17 +77,55 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
 
   private[spark] var scheduler: TaskScheduler = null
 
-  // executor ID -> timestamp of when the last heartbeat from this executor was received
+  /**
+   * [SPARK-39984]
+   * Please make sure the intersection between `executorLastSeen` and `waitingList` is an empty set.
+   * If the intersection is not empty, it is possible to never kill the executor until the executor
+   * recovers. When an executor is in both `executorLastSeen` and `waitingList`, the value of
+   * `workerLastHeartbeat` in waitingList may update if the worker sends heartbeats to master
+   * normally.
+   *
+   * `executorLastSeen`:
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from this executor was received
+   *
+   *  `waitingList`: executor ID -> WorkerLastHeartbeat
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from the worker was received
+   *
+   * when driver does not receive any heartbeat from an executor for `executorTimeoutMs` seconds,
+   * the driver will ask master for the last heartbeat from the worker which the executor is running
+   * on.
+   */
   private val executorLastSeen = new HashMap[String, Long]
+  private val waitingList = new HashMap[String, Long]
 
   private val executorTimeoutMs = sc.conf.get(
     config.STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT
-  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s"))
+  ).getOrElse(
+    sc.conf.get(Network.NETWORK_EXECUTOR_TIMEOUT) match {
+      case Some(executorTimeout) => Utils.timeStringAsMs(s"${executorTimeout}s")
+      case None => Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s")
+    }
+  )
 
   private val checkTimeoutIntervalMs = sc.conf.get(Network.NETWORK_TIMEOUT_INTERVAL)
 
   private val executorHeartbeatIntervalMs = sc.conf.get(config.EXECUTOR_HEARTBEAT_INTERVAL)
 
+  /**
+   * Currently, [SPARK-39984] is only for StandaloneSchedulerBackend.
+   *
+   * `checkWorkerLastHeartbeat`: A flag to enable two-phase executor timeout.
+   * `waitingListTimeout`: The timeout used for waitingList.
+   */
+  private val checkWorkerLastHeartbeat = sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT)
+  private val waitingListTimeout = checkWorkerLastHeartbeat match {
+    case true =>
+      sc.conf.get(Network.HEARTBEAT_WAITINGLIST_TIMEOUT).getOrElse(Utils.timeStringAsMs("30s"))
+    case false => Utils.timeStringAsMs("0s")
+  }
+

Review Comment:
   Updated.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r946374353


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +241,120 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            // TODO (SPARK-39984): Update causedByApp when we have a hanging task detector
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms")))
+          // LocalSchedulerBackend is used locally and only has one single executor
+          case _: LocalSchedulerBackend =>
+
+          case other => throw new UnsupportedOperationException(
+            s"Unknown scheduler backend: ${other.getClass}")
+        }
+      }
+    })
+  }
+
+  private def isStandalone(): Boolean = {
+    sc.schedulerBackend match {
+      case backend: StandaloneSchedulerBackend => true
+      case _ => false
+    }

Review Comment:
   Hi @Ngone51, thank you for your review! 
   
   Because `schedulerBackend` is possible to be None, I used pattern matching instead of `isInstanceOf`. @mridulm and I have discussed the issues above. I will tag you in these two discussion threads.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r946395124


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +241,120 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            // TODO (SPARK-39984): Update causedByApp when we have a hanging task detector
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms")))
+          // LocalSchedulerBackend is used locally and only has one single executor
+          case _: LocalSchedulerBackend =>
+
+          case other => throw new UnsupportedOperationException(
+            s"Unknown scheduler backend: ${other.getClass}")
+        }
+      }
+    })
+  }
+
+  private def isStandalone(): Boolean = {
+    sc.schedulerBackend match {
+      case backend: StandaloneSchedulerBackend => true
+      case _ => false
+    }
+  }
+
+  private def removeExecutorFromExpiryCandidates(executorId: String): Unit = {
+    if (checkWorkerLastHeartbeat && isStandalone()) {
+      executorExpiryCandidates.remove(executorId)
+    }
+  }
+
   private def expireDeadHosts(): Unit = {
+  /**
+   * [SPARK-39984]
+   * The driver’s HeartbeatReceiver will expire an executor if it does not receive any heartbeat
+   * from the executor for `executorTimeoutMs` (default 120s) seconds. However, lowering from 120
+   * seconds has other challenges. For example: when executor is performing full GC, it cannot
+   * send/reply any message for tens of seconds (based on your environment). Hence,
+   * HeartbeatReceiver cannot whether the heartbeat loss is caused by network issues or other
+   * reasons (e.g. full GC). To address this, we designed a new Heartbeat Receiver mechanism for
+   * standalone deployments.
+   *
+   * For standalone deployments:
+   * If driver does not receive any heartbeat from the executor for `executorTimeoutMs` seconds,
+   * HeartbeatReceiver will send a request to master to ask for the latest heartbeat from the
+   * worker which the executor runs on. HeartbeatReceiver can determine whether the heartbeat loss
+   * is caused by network issues or other issues (e.g. GC). If the heartbeat loss is not caused by
+   * network issues, the HeartbeatReceiver will put the executor into `executorExpiryCandidates`
+   * rather than expiring it immediately.
+   */
     logTrace("Checking for hosts with no recent heartbeats in HeartbeatReceiver.")
+    logWarning(s"Keep `expiryCandidatesTimeout` larger than `HEARTBEAT_MILLIS` in" +
+      s"deploy/worker/Worker.scala to know whether master lost any heartbeat from the" +
+      s"worker or not.")
     val now = clock.getTimeMillis()
-    for ((executorId, lastSeenMs) <- executorLastSeen) {
-      if (now - lastSeenMs > executorTimeoutMs) {
-        logWarning(s"Removing executor $executorId with no recent heartbeats: " +
-          s"${now - lastSeenMs} ms exceeds timeout $executorTimeoutMs ms")
-        // Asynchronously kill the executor to avoid blocking the current thread
-        killExecutorThread.submit(new Runnable {
-          override def run(): Unit = Utils.tryLogNonFatalError {
-            // Note: we want to get an executor back after expiring this one,
-            // so do not simply call `sc.killExecutor` here (SPARK-8119)
-            sc.killAndReplaceExecutor(executorId)
-            // SPARK-27348: in case of the executors which are not gracefully shut down,
-            // we should remove lost executors from CoarseGrainedSchedulerBackend manually
-            // here to guarantee two things:
-            // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
-            //    a lost executor instead of waiting for disconnect message
-            // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
-            //    those executors to avoid app hang
-            sc.schedulerBackend match {
-              case backend: CoarseGrainedSchedulerBackend =>
-                backend.driverEndpoint.send(RemoveExecutor(executorId,
-                  ExecutorProcessLost(
-                    s"Executor heartbeat timed out after ${now - lastSeenMs} ms")))
-
-              // LocalSchedulerBackend is used locally and only has one single executor
-              case _: LocalSchedulerBackend =>
-
-              case other => throw new UnsupportedOperationException(
-                s"Unknown scheduler backend: ${other.getClass}")
-            }
+    if (!checkWorkerLastHeartbeat || !isStandalone()) {
+      for ((executorId, lastSeenMs) <- executorLastSeen) {
+        if (now - lastSeenMs > executorTimeoutMs) {
+          killExecutor(executorId, now - lastSeenMs)
+          executorLastSeen.remove(executorId)
+        }
+      }
+    } else {
+      for ((executorId, workerLastHeartbeat) <- executorExpiryCandidates) {
+        if (now - workerLastHeartbeat > expiryCandidatesTimeout) {
+          killExecutor(executorId, now - workerLastHeartbeat)
+          executorExpiryCandidates.remove(executorId)
+          executorLastSeen.remove(executorId)
+        }
+      }
+
+      val buf = new ArrayBuffer[String]()
+      for ((executorId, lastSeenMs) <- executorLastSeen) {
+        if (now - lastSeenMs > executorTimeoutMs) {
+            buf += executorId
+        }
+      }
+
+      sc.schedulerBackend match {
+        case backend: StandaloneSchedulerBackend =>
+          backend.client.workerLastHeartbeat(sc.applicationId, buf) match {
+            case Some(workerLastHeartbeats) =>
+              for ((executorId, workerLastHeartbeat) <- buf zip workerLastHeartbeats) {
+                if (now - workerLastHeartbeat > expiryCandidatesTimeout) {
+                  val lastSeenMs = executorLastSeen.get(executorId).get

Review Comment:
   Updated https://github.com/apache/spark/pull/37411/commits/600f82a065aafff96c168fb9130b5eecb0dec4ff



##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +241,120 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            // TODO (SPARK-39984): Update causedByApp when we have a hanging task detector
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms")))
+          // LocalSchedulerBackend is used locally and only has one single executor
+          case _: LocalSchedulerBackend =>
+
+          case other => throw new UnsupportedOperationException(
+            s"Unknown scheduler backend: ${other.getClass}")
+        }
+      }
+    })
+  }
+
+  private def isStandalone(): Boolean = {
+    sc.schedulerBackend match {
+      case backend: StandaloneSchedulerBackend => true
+      case _ => false
+    }
+  }
+
+  private def removeExecutorFromExpiryCandidates(executorId: String): Unit = {
+    if (checkWorkerLastHeartbeat && isStandalone()) {
+      executorExpiryCandidates.remove(executorId)
+    }
+  }
+
   private def expireDeadHosts(): Unit = {
+  /**
+   * [SPARK-39984]
+   * The driver’s HeartbeatReceiver will expire an executor if it does not receive any heartbeat
+   * from the executor for `executorTimeoutMs` (default 120s) seconds. However, lowering from 120
+   * seconds has other challenges. For example: when executor is performing full GC, it cannot
+   * send/reply any message for tens of seconds (based on your environment). Hence,
+   * HeartbeatReceiver cannot whether the heartbeat loss is caused by network issues or other
+   * reasons (e.g. full GC). To address this, we designed a new Heartbeat Receiver mechanism for
+   * standalone deployments.
+   *
+   * For standalone deployments:
+   * If driver does not receive any heartbeat from the executor for `executorTimeoutMs` seconds,
+   * HeartbeatReceiver will send a request to master to ask for the latest heartbeat from the
+   * worker which the executor runs on. HeartbeatReceiver can determine whether the heartbeat loss
+   * is caused by network issues or other issues (e.g. GC). If the heartbeat loss is not caused by
+   * network issues, the HeartbeatReceiver will put the executor into `executorExpiryCandidates`
+   * rather than expiring it immediately.
+   */
     logTrace("Checking for hosts with no recent heartbeats in HeartbeatReceiver.")
+    logWarning(s"Keep `expiryCandidatesTimeout` larger than `HEARTBEAT_MILLIS` in" +
+      s"deploy/worker/Worker.scala to know whether master lost any heartbeat from the" +
+      s"worker or not.")
     val now = clock.getTimeMillis()
-    for ((executorId, lastSeenMs) <- executorLastSeen) {
-      if (now - lastSeenMs > executorTimeoutMs) {
-        logWarning(s"Removing executor $executorId with no recent heartbeats: " +
-          s"${now - lastSeenMs} ms exceeds timeout $executorTimeoutMs ms")
-        // Asynchronously kill the executor to avoid blocking the current thread
-        killExecutorThread.submit(new Runnable {
-          override def run(): Unit = Utils.tryLogNonFatalError {
-            // Note: we want to get an executor back after expiring this one,
-            // so do not simply call `sc.killExecutor` here (SPARK-8119)
-            sc.killAndReplaceExecutor(executorId)
-            // SPARK-27348: in case of the executors which are not gracefully shut down,
-            // we should remove lost executors from CoarseGrainedSchedulerBackend manually
-            // here to guarantee two things:
-            // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
-            //    a lost executor instead of waiting for disconnect message
-            // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
-            //    those executors to avoid app hang
-            sc.schedulerBackend match {
-              case backend: CoarseGrainedSchedulerBackend =>
-                backend.driverEndpoint.send(RemoveExecutor(executorId,
-                  ExecutorProcessLost(
-                    s"Executor heartbeat timed out after ${now - lastSeenMs} ms")))
-
-              // LocalSchedulerBackend is used locally and only has one single executor
-              case _: LocalSchedulerBackend =>
-
-              case other => throw new UnsupportedOperationException(
-                s"Unknown scheduler backend: ${other.getClass}")
-            }
+    if (!checkWorkerLastHeartbeat || !isStandalone()) {
+      for ((executorId, lastSeenMs) <- executorLastSeen) {
+        if (now - lastSeenMs > executorTimeoutMs) {
+          killExecutor(executorId, now - lastSeenMs)
+          executorLastSeen.remove(executorId)
+        }
+      }
+    } else {
+      for ((executorId, workerLastHeartbeat) <- executorExpiryCandidates) {
+        if (now - workerLastHeartbeat > expiryCandidatesTimeout) {
+          killExecutor(executorId, now - workerLastHeartbeat)
+          executorExpiryCandidates.remove(executorId)
+          executorLastSeen.remove(executorId)
+        }
+      }
+
+      val buf = new ArrayBuffer[String]()
+      for ((executorId, lastSeenMs) <- executorLastSeen) {
+        if (now - lastSeenMs > executorTimeoutMs) {
+            buf += executorId
+        }
+      }
+
+      sc.schedulerBackend match {
+        case backend: StandaloneSchedulerBackend =>
+          backend.client.workerLastHeartbeat(sc.applicationId, buf) match {
+            case Some(workerLastHeartbeats) =>
+              for ((executorId, workerLastHeartbeat) <- buf zip workerLastHeartbeats) {
+                if (now - workerLastHeartbeat > expiryCandidatesTimeout) {
+                  val lastSeenMs = executorLastSeen.get(executorId).get
+                  killExecutor(executorId, now - lastSeenMs)
+                  executorExpiryCandidates.remove(executorId)
+                } else {
+                  executorExpiryCandidates(executorId) = workerLastHeartbeat
+                }
+                executorLastSeen.remove(executorId)
+              }
+            case None =>
+              for (executorId <- buf) {
+                val lastSeenMs = executorLastSeen.get(executorId).get

Review Comment:
   Updated https://github.com/apache/spark/pull/37411/commits/600f82a065aafff96c168fb9130b5eecb0dec4ff



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r946373638


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -77,17 +77,44 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
 
   private[spark] var scheduler: TaskScheduler = null
 
-  // executor ID -> timestamp of when the last heartbeat from this executor was received
+  /**
+   * [SPARK-39984]
+   * Please make sure the intersection between `executorLastSeen` and `waitingList` is an empty set.
+   * If the intersection is not empty, it is possible to never kill the executor until the executor
+   * recovers. When an executor is in both `executorLastSeen` and `waitingList`, the value of
+   * `workerLastHeartbeat` in waitingList may update if the worker sends heartbeats to master
+   * normally.
+   *
+   * `executorLastSeen`:
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from this executor was received
+   *
+   *  `waitingList`: executor ID -> WorkerLastHeartbeat
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from the worker was received
+   *
+   * when driver does not receive any heartbeat from an executor for `executorTimeoutMs` seconds,
+   * the driver will ask master for the last heartbeat from the worker which the executor is running
+   * on.
+   */
   private val executorLastSeen = new HashMap[String, Long]
+  private val waitingList = new HashMap[String, Long]
 
   private val executorTimeoutMs = sc.conf.get(
     config.STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT
-  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s"))
+  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_EXECUTOR_TIMEOUT)}s"))
 
   private val checkTimeoutIntervalMs = sc.conf.get(Network.NETWORK_TIMEOUT_INTERVAL)
 
   private val executorHeartbeatIntervalMs = sc.conf.get(config.EXECUTOR_HEARTBEAT_INTERVAL)
 
+  /**
+   * Currently, [SPARK-39984] is only for StandaloneSchedulerBackend.
+   */
+  private val checkWorkerLastHeartbeat =
+    sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT) &&
+      sc.schedulerBackend.isInstanceOf[StandaloneSchedulerBackend]

Review Comment:
   @Ngone51 Here is the other discussion about the lack of scheduler backend.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] mridulm commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
mridulm commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r942900759


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -77,17 +77,44 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
 
   private[spark] var scheduler: TaskScheduler = null
 
-  // executor ID -> timestamp of when the last heartbeat from this executor was received
+  /**
+   * [SPARK-39984]
+   * Please make sure the intersection between `executorLastSeen` and `waitingList` is an empty set.
+   * If the intersection is not empty, it is possible to never kill the executor until the executor
+   * recovers. When an executor is in both `executorLastSeen` and `waitingList`, the value of
+   * `workerLastHeartbeat` in waitingList may update if the worker sends heartbeats to master
+   * normally.
+   *
+   * `executorLastSeen`:
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from this executor was received
+   *
+   *  `waitingList`: executor ID -> WorkerLastHeartbeat
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from the worker was received
+   *
+   * when driver does not receive any heartbeat from an executor for `executorTimeoutMs` seconds,
+   * the driver will ask master for the last heartbeat from the worker which the executor is running
+   * on.
+   */
   private val executorLastSeen = new HashMap[String, Long]
+  private val waitingList = new HashMap[String, Long]
 
   private val executorTimeoutMs = sc.conf.get(
     config.STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT
-  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s"))
+  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_EXECUTOR_TIMEOUT)}s"))
 
   private val checkTimeoutIntervalMs = sc.conf.get(Network.NETWORK_TIMEOUT_INTERVAL)
 
   private val executorHeartbeatIntervalMs = sc.conf.get(config.EXECUTOR_HEARTBEAT_INTERVAL)
 
+  /**
+   * Currently, [SPARK-39984] is only for StandaloneSchedulerBackend.
+   */
+  private val checkWorkerLastHeartbeat =
+    sc.conf.get(HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT) &&
+      sc.schedulerBackend.isInstanceOf[StandaloneSchedulerBackend]

Review Comment:
   Note that you can actually have `ExecutorRegistered` before `sc.schedulerBackend` is set - this is essentially a race condition. For example, we have a `if (scheduler != null) {` check while handling `Heartbeat`.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] mridulm commented on pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
mridulm commented on PR #37411:
URL: https://github.com/apache/spark/pull/37411#issuecomment-1213415203

   Btw, any thoughts on this ?
   > Are the changes here necessarily only for standalone ? Why not k8s and yarn ?
   
   The changes are specific to standalone - but the idea should be transferable to others as well, right ?


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] github-actions[bot] commented on pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #37411:
URL: https://github.com/apache/spark/pull/37411#issuecomment-1353933005

   We're closing this PR because it hasn't been updated in a while. This isn't a judgement on the merit of the PR in any way. It's just a way of keeping the PR queue manageable.
   If you'd like to revive this PR, please reopen it and ask a committer to remove the Stale tag!


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r963377849


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -77,17 +77,61 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
 
   private[spark] var scheduler: TaskScheduler = null
 
-  // executor ID -> timestamp of when the last heartbeat from this executor was received
+  /**
+   * [SPARK-39984]
+   * Please make sure the intersection between `executorLastSeen` and `executorExpiryCandidates` is
+   * an empty set. If the intersection is not empty, it is possible to never kill the executor until
+   * the executor recovers. When an executor is in both `executorLastSeen` and
+   * `executorExpiryCandidates`, the value of `workerLastHeartbeat` in `executorExpiryCandidates`
+   * may update if the worker sends heartbeats to master normally.
+   *
+   * `executorLastSeen`:
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from this executor was received
+   *
+   * `executorExpiryCandidates`: executor ID -> WorkerLastHeartbeat
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from the worker was received
+   *
+   * when driver does not receive any heartbeat from an executor for `executorTimeoutMs` seconds,
+   * the driver will ask master for the last heartbeat from the worker which the executor is running
+   * on.
+   */
   private val executorLastSeen = new HashMap[String, Long]
+  private val executorExpiryCandidates = new HashMap[String, Long]
 
   private val executorTimeoutMs = sc.conf.get(
     config.STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT
-  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s"))
+  ).getOrElse(
+    sc.conf.get(Network.NETWORK_EXECUTOR_TIMEOUT) match {
+      case Some(executorTimeout) => executorTimeout
+      case None => Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s")
+    }
+  )
 
   private val checkTimeoutIntervalMs = sc.conf.get(Network.NETWORK_TIMEOUT_INTERVAL)
 
   private val executorHeartbeatIntervalMs = sc.conf.get(config.EXECUTOR_HEARTBEAT_INTERVAL)
 
+  /**
+   * Currently, [SPARK-39984] is only for StandaloneSchedulerBackend.
+   *
+   * `checkWorkerLastHeartbeat`: A flag to enable two-phase executor timeout.
+   * `expiryCandidatesTimeout`: The timeout used for executorExpiryCandidates.
+   */
+  private val checkWorkerLastHeartbeat = {

Review Comment:
   Updated. 
   
   (1) Declare the variable `checkWorkerLastHeartbeat` as a lazy variable
   (2) Set an initial delay for `ExpireDeadHosts`
   (3) If the scheduler backend has not initialized when `checkWorkerLastHeartbeat` is evaluated, an `UnsupportedOperationException` will be thrown.
   ```scala
   private lazy val checkWorkerLastHeartbeat = sc.schedulerBackend match {
     case _: CoarseGrainedSchedulerBackend =>
       sc.conf.get(config.HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT) &&
         sc.schedulerBackend.isInstanceOf[StandaloneSchedulerBackend]
     case _: LocalSchedulerBackend => false
     case other => throw new UnsupportedOperationException(
       s"Unknown scheduler backend: ${other.getClass}")
   }
   ```



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on PR #37411:
URL: https://github.com/apache/spark/pull/37411#issuecomment-1237583132

   TODO: NETWORK_TIMEOUT_INTERVAL (15s)


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r961328575


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -77,17 +77,61 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
 
   private[spark] var scheduler: TaskScheduler = null
 
-  // executor ID -> timestamp of when the last heartbeat from this executor was received
+  /**
+   * [SPARK-39984]
+   * Please make sure the intersection between `executorLastSeen` and `executorExpiryCandidates` is
+   * an empty set. If the intersection is not empty, it is possible to never kill the executor until
+   * the executor recovers. When an executor is in both `executorLastSeen` and
+   * `executorExpiryCandidates`, the value of `workerLastHeartbeat` in `executorExpiryCandidates`
+   * may update if the worker sends heartbeats to master normally.
+   *
+   * `executorLastSeen`:
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from this executor was received
+   *
+   * `executorExpiryCandidates`: executor ID -> WorkerLastHeartbeat
+   *  - key: executor ID
+   *  - value: timestamp of when the last heartbeat from the worker was received
+   *
+   * when driver does not receive any heartbeat from an executor for `executorTimeoutMs` seconds,
+   * the driver will ask master for the last heartbeat from the worker which the executor is running
+   * on.
+   */
   private val executorLastSeen = new HashMap[String, Long]
+  private val executorExpiryCandidates = new HashMap[String, Long]
 
   private val executorTimeoutMs = sc.conf.get(
     config.STORAGE_BLOCKMANAGER_HEARTBEAT_TIMEOUT
-  ).getOrElse(Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s"))
+  ).getOrElse(
+    sc.conf.get(Network.NETWORK_EXECUTOR_TIMEOUT) match {
+      case Some(executorTimeout) => executorTimeout
+      case None => Utils.timeStringAsMs(s"${sc.conf.get(Network.NETWORK_TIMEOUT)}s")
+    }
+  )
 
   private val checkTimeoutIntervalMs = sc.conf.get(Network.NETWORK_TIMEOUT_INTERVAL)
 
   private val executorHeartbeatIntervalMs = sc.conf.get(config.EXECUTOR_HEARTBEAT_INTERVAL)
 
+  /**
+   * Currently, [SPARK-39984] is only for StandaloneSchedulerBackend.
+   *
+   * `checkWorkerLastHeartbeat`: A flag to enable two-phase executor timeout.
+   * `expiryCandidatesTimeout`: The timeout used for executorExpiryCandidates.
+   */
+  private val checkWorkerLastHeartbeat = {

Review Comment:
   Let me clarify the definition of "initial delay". In my understanding, the "initial delay" above means the second argument of `scheduleAtFixedRate` ([Link](https://sourcegraph.com/github.com/apache/spark@b012cb7/-/blob/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala?L107)). Is it correct?
   
   ```scala
   override def onStart(): Unit = {
     timeoutCheckingTask = eventLoopThread.scheduleAtFixedRate(
     () => Utils.tryLogNonFatalError { Option(self).foreach(_.ask[Boolean](ExpireDeadHosts)) },
     0, checkTimeoutIntervalMs, TimeUnit.MILLISECONDS)
   }
   ```
   
   I agreed with you that initial delay can significantly decrease the probability of first expireDeadHosts trigger happening before scheduler backend initialization. However, it cannot totally prevent this edge case. Hence, my thought is to define
   `checkWorkerLastHeartbeat` as a function, and we can totally prevent the edge case. Does it 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r945367796


##########
core/src/main/scala/org/apache/spark/internal/config/Network.scala:
##########
@@ -51,6 +51,21 @@ private[spark] object Network {
       .timeConf(TimeUnit.MILLISECONDS)
       .createWithDefaultString(STORAGE_BLOCKMANAGER_TIMEOUTINTERVAL.defaultValueString)
 
+  private[spark] val NETWORK_EXECUTOR_TIMEOUT =
+    ConfigBuilder("spark.network.executorTimeout")
+      .version("3.4.0")
+      .timeConf(TimeUnit.MILLISECONDS)
+      .createOptional
+
+  private[spark] val HEARTBEAT_EXPIRY_CANDIDATES_TIMEOUT =
+    ConfigBuilder("spark.network.expiryCandidatesTimeout")
+      .doc("This config is a timeout used for heartbeat receiver `executorExpiryCandidates`. Be" +
+        "effective only when HEARTBEAT_RECEIVER_CHECK_WORKER_LAST_HEARTBEAT is enabled. See" +
+        "[SPARK-39984] for more details")
+      .version("3.4.0")
+      .timeConf(TimeUnit.MILLISECONDS)
+      .createWithDefaultString("30s")

Review Comment:
   +1. 
   
   We can add this config to the namespace `spark.executor.heartbeat`. cc @kevin85421 



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kevin85421 commented on a diff in pull request #37411: [SPARK-39984][CORE] Check workerLastHeartbeat with master before HeartbeatReceiver expires an executor

Posted by GitBox <gi...@apache.org>.
kevin85421 commented on code in PR #37411:
URL: https://github.com/apache/spark/pull/37411#discussion_r947199549


##########
core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala:
##########
@@ -199,41 +241,120 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
     removeExecutor(executorRemoved.executorId)
   }
 
+  private def killExecutor(executorId: String, timeout: Long): Unit = {
+    logWarning(s"Removing executor $executorId with no recent heartbeats: " +
+      s"${timeout} ms exceeds timeout $executorTimeoutMs ms")
+    killExecutorThread.submit(new Runnable {
+      override def run(): Unit = Utils.tryLogNonFatalError {
+        // Note: we want to get an executor back after expiring this one,
+        // so do not simply call `sc.killExecutor` here (SPARK-8119)
+        sc.killAndReplaceExecutor(executorId)
+        // SPARK-27348: in case of the executors which are not gracefully shut down,
+        // we should remove lost executors from CoarseGrainedSchedulerBackend manually
+        // here to guarantee two things:
+        // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
+        //    a lost executor instead of waiting for disconnect message
+        // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
+        //    those executors to avoid app hang
+        sc.schedulerBackend match {
+          case backend: CoarseGrainedSchedulerBackend =>
+            // TODO (SPARK-39984): Update causedByApp when we have a hanging task detector
+            backend.driverEndpoint.send(RemoveExecutor(executorId,
+              ExecutorProcessLost(
+                s"Executor heartbeat timed out after ${timeout} ms")))
+          // LocalSchedulerBackend is used locally and only has one single executor
+          case _: LocalSchedulerBackend =>
+
+          case other => throw new UnsupportedOperationException(
+            s"Unknown scheduler backend: ${other.getClass}")
+        }
+      }
+    })
+  }
+
+  private def isStandalone(): Boolean = {
+    sc.schedulerBackend match {
+      case backend: StandaloneSchedulerBackend => true
+      case _ => false
+    }
+  }
+
+  private def removeExecutorFromExpiryCandidates(executorId: String): Unit = {
+    if (checkWorkerLastHeartbeat && isStandalone()) {
+      executorExpiryCandidates.remove(executorId)
+    }
+  }
+
   private def expireDeadHosts(): Unit = {
+  /**
+   * [SPARK-39984]
+   * The driver’s HeartbeatReceiver will expire an executor if it does not receive any heartbeat
+   * from the executor for `executorTimeoutMs` (default 120s) seconds. However, lowering from 120
+   * seconds has other challenges. For example: when executor is performing full GC, it cannot
+   * send/reply any message for tens of seconds (based on your environment). Hence,
+   * HeartbeatReceiver cannot whether the heartbeat loss is caused by network issues or other
+   * reasons (e.g. full GC). To address this, we designed a new Heartbeat Receiver mechanism for
+   * standalone deployments.
+   *
+   * For standalone deployments:
+   * If driver does not receive any heartbeat from the executor for `executorTimeoutMs` seconds,
+   * HeartbeatReceiver will send a request to master to ask for the latest heartbeat from the
+   * worker which the executor runs on. HeartbeatReceiver can determine whether the heartbeat loss
+   * is caused by network issues or other issues (e.g. GC). If the heartbeat loss is not caused by
+   * network issues, the HeartbeatReceiver will put the executor into `executorExpiryCandidates`
+   * rather than expiring it immediately.
+   */
     logTrace("Checking for hosts with no recent heartbeats in HeartbeatReceiver.")
+    logWarning(s"Keep `expiryCandidatesTimeout` larger than `HEARTBEAT_MILLIS` in" +
+      s"deploy/worker/Worker.scala to know whether master lost any heartbeat from the" +
+      s"worker or not.")
     val now = clock.getTimeMillis()
-    for ((executorId, lastSeenMs) <- executorLastSeen) {
-      if (now - lastSeenMs > executorTimeoutMs) {
-        logWarning(s"Removing executor $executorId with no recent heartbeats: " +
-          s"${now - lastSeenMs} ms exceeds timeout $executorTimeoutMs ms")
-        // Asynchronously kill the executor to avoid blocking the current thread
-        killExecutorThread.submit(new Runnable {
-          override def run(): Unit = Utils.tryLogNonFatalError {
-            // Note: we want to get an executor back after expiring this one,
-            // so do not simply call `sc.killExecutor` here (SPARK-8119)
-            sc.killAndReplaceExecutor(executorId)
-            // SPARK-27348: in case of the executors which are not gracefully shut down,
-            // we should remove lost executors from CoarseGrainedSchedulerBackend manually
-            // here to guarantee two things:
-            // 1) explicitly remove executor information from CoarseGrainedSchedulerBackend for
-            //    a lost executor instead of waiting for disconnect message
-            // 2) call scheduler.executorLost() underlying to fail any tasks assigned to
-            //    those executors to avoid app hang
-            sc.schedulerBackend match {
-              case backend: CoarseGrainedSchedulerBackend =>
-                backend.driverEndpoint.send(RemoveExecutor(executorId,
-                  ExecutorProcessLost(
-                    s"Executor heartbeat timed out after ${now - lastSeenMs} ms")))
-
-              // LocalSchedulerBackend is used locally and only has one single executor
-              case _: LocalSchedulerBackend =>
-
-              case other => throw new UnsupportedOperationException(
-                s"Unknown scheduler backend: ${other.getClass}")
-            }
+    if (!checkWorkerLastHeartbeat || !isStandalone()) {
+      for ((executorId, lastSeenMs) <- executorLastSeen) {
+        if (now - lastSeenMs > executorTimeoutMs) {
+          killExecutor(executorId, now - lastSeenMs)
+          executorLastSeen.remove(executorId)
+        }
+      }
+    } else {
+      for ((executorId, workerLastHeartbeat) <- executorExpiryCandidates) {
+        if (now - workerLastHeartbeat > expiryCandidatesTimeout) {
+          killExecutor(executorId, now - workerLastHeartbeat)
+          executorExpiryCandidates.remove(executorId)
+          executorLastSeen.remove(executorId)
+        }
+      }

Review Comment:
   Updated https://github.com/apache/spark/pull/37411/commits/e9fe8d1db2bc5307a437e903d0991c3f4c7d1949
   
   There is a slight difference when `backend.client.workerLastHeartbeat` matches the case `None`. I set `executorExpiryCandidates(executorId)` to `lastSeenMs` instead of `Long.Max`.



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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