Github user vanzin commented on a diff in the pull request:
https://github.com/apache/spark/pull/8007#discussion_r37355580
--- Diff:
core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala
---
@@ -91,6 +92,66 @@ private[spark] abstract class YarnSchedulerBackend(
}
/**
+ * Override the DriverEndpoint to add extra logic for the case when an
executor is disconnected.
+ * We should check the cluster manager and find if the loss of the
executor was caused by YARN
+ * force killing it due to preemption.
+ */
+ private class YarnDriverEndpoint(rpcEnv: RpcEnv, sparkProperties:
ArrayBuffer[(String, String)])
+ extends DriverEndpoint(rpcEnv, sparkProperties) {
+
+ private val pendingDisconnectedExecutors = new HashSet[String]
+ private val handleDisconnectedExecutorThreadPool =
+
ThreadUtils.newDaemonCachedThreadPool("yarn-driver-handle-lost-executor-thread-pool")
+
+ /**
+ * When onDisconnected is received at the driver endpoint, the
superclass DriverEndpoint
+ * handles it by assuming the Executor was lost for a bad reason and
removes the executor
+ * immediately.
+ *
+ * In YARN's case however it is crucial to talk to the application
master and ask why the
+ * executor had exited. In particular, the executor may have exited
due to the executor
+ * having been preempted. If the executor "exited normally" according
to the application
+ * master then we pass that information down to the TaskSetManager to
inform the
+ * TaskSetManager that tasks on that lost executor should not count
towards a job failure.
+ */
+ override def onDisconnected(rpcAddress: RpcAddress): Unit = {
--- End diff --
One side-effect of this is that there will be a delay between the executor
disconnecting from the driver and it being marked as unavailable for running
tasks. So isn't it possible that while we're waiting for the AM to reply, the
scheduler will try to run tasks on that executor?
Is there some sort of "soft unregistration" that could be done here so that
the executor is not used for new tasks, but we still haven't failed the
existing tasks pending figuring out the exit reason?
---
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]