Github user sryza commented on a diff in the pull request:
https://github.com/apache/spark/pull/4363#discussion_r24212764
--- Diff: core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala ---
@@ -17,33 +17,85 @@
package org.apache.spark
-import akka.actor.Actor
+import scala.concurrent.duration._
+import scala.collection.mutable
+
+import akka.actor.{Actor, Cancellable}
+
import org.apache.spark.executor.TaskMetrics
import org.apache.spark.storage.BlockManagerId
-import org.apache.spark.scheduler.TaskScheduler
+import org.apache.spark.scheduler.{SlaveLost, TaskScheduler}
import org.apache.spark.util.ActorLogReceive
/**
* A heartbeat from executors to the driver. This is a shared message used
by several internal
- * components to convey liveness or execution information for in-progress
tasks.
+ * components to convey liveness or execution information for in-progress
tasks. It will also
+ * expiry the hosts that have no heartbeat for more than
spark.executor.heartbeat.timeoutMs.
*/
private[spark] case class Heartbeat(
executorId: String,
taskMetrics: Array[(Long, TaskMetrics)], // taskId -> TaskMetrics
blockManagerId: BlockManagerId)
+private[spark] case object ExpireDeadHosts
+
private[spark] case class HeartbeatResponse(reregisterBlockManager:
Boolean)
/**
* Lives in the driver to receive heartbeats from executors..
*/
-private[spark] class HeartbeatReceiver(scheduler: TaskScheduler)
+private[spark] class HeartbeatReceiver(sc: SparkContext, scheduler:
TaskScheduler)
extends Actor with ActorLogReceive with Logging {
+ val executorLastSeen = new mutable.HashMap[String, Long]
+
+ val slaveTimeout = sc.conf.getLong("spark.executor.heartbeat.timeoutMs",
120 * 1000)
--- End diff --
Can we call these spark.driver.executorTimeoutMs and
spark.driver.executorTimeoutIntervalMs?
Also, they should be updated in the configuration, and for backwards
compatibility, if not set, they should default to the values of
spark.storage.blockManagerSlaveTimeoutMs and
spark.storage.blockManagerSlaveTimeoutIntervalMs
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]