-
Notifications
You must be signed in to change notification settings - Fork 0
Implement Automatic Killing of Blacklisted Executors #2
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: SPARK-16654
Are you sure you want to change the base?
Changes from all commits
414f99f
719f93c
2566ce7
44cf4d8
63da9a3
f4ad62d
4f4d21d
3140017
e605eaf
4fb6b12
322a232
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -21,7 +21,7 @@ import java.util.concurrent.atomic.AtomicReference | |
|
|
||
| import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} | ||
|
|
||
| import org.apache.spark.{SparkConf, SparkContext} | ||
| import org.apache.spark.{ExecutorAllocationClient, SparkConf, SparkContext} | ||
| import org.apache.spark.internal.Logging | ||
| import org.apache.spark.internal.config | ||
| import org.apache.spark.util.{Clock, SystemClock, Utils} | ||
|
|
@@ -50,10 +50,11 @@ import org.apache.spark.util.{Clock, SystemClock, Utils} | |
| private[scheduler] class BlacklistTracker ( | ||
| private val listenerBus: LiveListenerBus, | ||
| conf: SparkConf, | ||
| scheduler: Option[ExecutorAllocationClient], | ||
| clock: Clock = new SystemClock()) extends Logging { | ||
|
|
||
| def this(sc: SparkContext) = { | ||
| this(sc.listenerBus, sc.getConf) | ||
| def this(sc: SparkContext, scheduler: Option[ExecutorAllocationClient]) = { | ||
| this(sc.listenerBus, sc.getConf, scheduler) | ||
| } | ||
|
|
||
| BlacklistTracker.validateBlacklistConfs(conf) | ||
|
|
@@ -169,6 +170,21 @@ private[scheduler] class BlacklistTracker ( | |
| if (newTotal >= MAX_FAILURES_PER_EXEC && !executorIdToBlacklistStatus.contains(exec)) { | ||
| logInfo(s"Blacklisting executor id: $exec because it has $newTotal" + | ||
| s" task failures in successful task sets") | ||
| conf.get(config.BLACKLIST_ENABLED) match { | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
|
||
| case Some(enabled) => | ||
| if (enabled) { | ||
| scheduler match { | ||
| case Some(scheduler) => | ||
| logInfo(s"Killing blacklisted executor id $exec" + | ||
| s"since spark.blacklist.kill is set.") | ||
| scheduler.killExecutors(Seq(exec), true, true) | ||
| case None => | ||
| logWarning(s"Not attempting to kill blacklisted executor id $exec" + | ||
| s"since scheduler is not defined.") | ||
| } | ||
| } | ||
| case None => | ||
| } | ||
| val node = failuresInTaskSet.node | ||
| executorIdToBlacklistStatus.put(exec, BlacklistedExecutor(node, expiryTime)) | ||
| listenerBus.post( | ||
|
|
@@ -183,6 +199,21 @@ private[scheduler] class BlacklistTracker ( | |
| if (blacklistedExecsOnNode.size >= MAX_FAILED_EXEC_PER_NODE) { | ||
| logInfo(s"Blacklisting node $node because it has ${blacklistedExecsOnNode.size} " + | ||
| s"executors blacklisted: ${blacklistedExecsOnNode}") | ||
| conf.get(config.BLACKLIST_ENABLED) match { | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
|
||
| case Some(enabled) => | ||
| if (enabled) { | ||
| scheduler match { | ||
| case Some(scheduler) => | ||
| logInfo(s"Killing all executors on blacklisted host $node" + | ||
| s"since spark.blacklist.kill is set.") | ||
| scheduler.killExecutorsOnHost(node) | ||
| case None => | ||
| logWarning(s"Not attempting to kill executors on blacklisted host $node" + | ||
| s"since scheduler is not defined.") | ||
| } | ||
| } | ||
| case None => | ||
| } | ||
| nodeIdToBlacklistExpiryTime.put(node, expiryTime) | ||
| listenerBus.post(SparkListenerNodeBlacklisted(now, node, blacklistedExecsOnNode.size)) | ||
| _nodeBlacklist.set(nodeIdToBlacklistExpiryTime.keySet.toSet) | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -703,7 +703,11 @@ private[spark] object TaskSchedulerImpl { | |
|
|
||
| private def maybeCreateBlacklistTracker(sc: SparkContext): Option[BlacklistTracker] = { | ||
| if (BlacklistTracker.isBlacklistEnabled(sc.conf)) { | ||
| Some(new BlacklistTracker(sc)) | ||
| val executorAllocClient: Option[ExecutorAllocationClient] = sc.schedulerBackend match { | ||
| case b: ExecutorAllocationClient => Some(b.asInstanceOf[ExecutorAllocationClient]) | ||
| case _ => None | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Maybe its best to just fail fast right here if blacklist.kill is enabled, but you don't have an |
||
| } | ||
| Some(new BlacklistTracker(sc, executorAllocClient)) | ||
| } else { | ||
| None | ||
| } | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -145,6 +145,11 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp | |
| // Ignoring the task kill since the executor is not registered. | ||
| logWarning(s"Attempted to kill task $taskId for unknown executor $executorId.") | ||
| } | ||
|
|
||
| case KillExecutorsOnHost(host) => | ||
| scheduler.getExecutorsAliveOnHost(host).foreach(exec => | ||
| killExecutors(exec.toSeq, replace = true, force = true) | ||
| ) | ||
| } | ||
|
|
||
| override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { | ||
|
|
@@ -547,7 +552,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp | |
| * @return whether the kill request is acknowledged. If list to kill is empty, it will return | ||
| * false. | ||
| */ | ||
| final def killExecutors( | ||
| final override def killExecutors( | ||
| executorIds: Seq[String], | ||
| replace: Boolean, | ||
| force: Boolean): Seq[String] = { | ||
|
|
@@ -603,6 +608,15 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp | |
| */ | ||
| protected def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = | ||
| Future.successful(false) | ||
|
|
||
| /** | ||
| * Request that the cluster manager kill all executors on a given host. | ||
| * @return whether the kill request is acknowledged | ||
| */ | ||
| final override def killExecutorsOnHost(host: String): Unit = { | ||
| logInfo(s"Requesting to kill any and all executors on host ${host}") | ||
| driverEndpoint.send(KillExecutorsOnHost(host)) | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'd include a comment here on why you do delegate this to driver endpoint, rather than doing it immediately here, something along the lines of: We have to be careful that there isn't a race between killing all executors on the bad host, and another executor getting registered on the same host. We do that by doing it within the DriverEndpoint, which is guaranteed to handle one message at a time since its a ThreadSafeRPCEndpoint |
||
| } | ||
| } | ||
|
|
||
| private[spark] object CoarseGrainedSchedulerBackend { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -466,6 +466,23 @@ class StandaloneDynamicAllocationSuite | |
| } | ||
| } | ||
|
|
||
| test("kill all executors on localhost") { | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. this doesn't really have anything to do with Dynamic Allocation, so this is a strange suite to put this test in. Though it does seem to have a useful framework -- perhaps there is a base class to pull out?
Owner
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'll refactor these tests when I start adding more tests for configuration parameters and the like. |
||
| sc = new SparkContext(appConf) | ||
| val appId = sc.applicationId | ||
| eventually(timeout(10.seconds), interval(10.millis)) { | ||
| val apps = getApplications() | ||
| assert(apps.size === 1) | ||
| assert(apps.head.id === appId) | ||
| assert(apps.head.executors.size === 2) | ||
| assert(apps.head.getExecutorLimit === Int.MaxValue) | ||
| } | ||
| // kill all executors | ||
| assert(killExecutorsOnHost(sc, "localhost").size == 2) | ||
| var apps = getApplications() | ||
| assert(apps.head.executors.size === 0) | ||
| assert(apps.head.getExecutorLimit === 0) | ||
| } | ||
|
|
||
| // =============================== | ||
| // | Utility methods for testing | | ||
| // =============================== | ||
|
|
@@ -527,6 +544,16 @@ class StandaloneDynamicAllocationSuite | |
| } | ||
| } | ||
|
|
||
| /** Kill the executors on a given host. */ | ||
| private def killExecutorsOnHost(sc: SparkContext, host: String): Seq[String] = { | ||
| syncExecutors(sc) | ||
| sc.schedulerBackend match { | ||
| case b: CoarseGrainedSchedulerBackend => | ||
| b.killExecutorsOnHost(host) | ||
| case _ => fail("expected coarse grained scheduler") | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Return a list of executor IDs belonging to this application. | ||
| * | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't think you need this in the api, do you?