Skip to content

Commit 9ff03fa

Browse files
committed
[SPARK-18553][CORE][BRANCH-2.0] Fix leak of TaskSetManager following executor loss
## What changes were proposed in this pull request? This patch fixes a critical resource leak in the TaskScheduler which could cause RDDs and ShuffleDependencies to be kept alive indefinitely if an executor with running tasks is permanently lost and the associated stage fails. This problem was originally identified by analyzing the heap dump of a driver belonging to a cluster that had run out of shuffle space. This dump contained several `ShuffleDependency` instances that were retained by `TaskSetManager`s inside the scheduler but were not otherwise referenced. Each of these `TaskSetManager`s was considered a "zombie" but had no running tasks and therefore should have been cleaned up. However, these zombie task sets were still referenced by the `TaskSchedulerImpl.taskIdToTaskSetManager` map. Entries are added to the `taskIdToTaskSetManager` map when tasks are launched and are removed inside of `TaskScheduler.statusUpdate()`, which is invoked by the scheduler backend while processing `StatusUpdate` messages from executors. The problem with this design is that a completely dead executor will never send a `StatusUpdate`. There is [some code](https://github.com/apache/spark/blob/072f4c518cdc57d705beec6bcc3113d9a6740819/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala#L338) in `statusUpdate` which handles tasks that exit with the `TaskState.LOST` state (which is supposed to correspond to a task failure triggered by total executor loss), but this state only seems to be used in Mesos fine-grained mode. There doesn't seem to be any code which performs per-task state cleanup for tasks that were running on an executor that completely disappears without sending any sort of final death message. The `executorLost` and [`removeExecutor`](https://github.com/apache/spark/blob/072f4c518cdc57d705beec6bcc3113d9a6740819/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala#L527) methods don't appear to perform any cleanup of the `taskId -> *` mappings, causing the leaks observed here. This patch's fix is to maintain a `executorId -> running task id` mapping so that these `taskId -> *` maps can be properly cleaned up following an executor loss. There are some potential corner-case interactions that I'm concerned about here, especially some details in [the comment](https://github.com/apache/spark/blob/072f4c518cdc57d705beec6bcc3113d9a6740819/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala#L523) in `removeExecutor`, so I'd appreciate a very careful review of these changes. This PR is opened against branch-2.0, where I first observed this problem, but will also need to be fixed in master, branch-2.1, and branch-1.6 (which I'll do in followup PRs after this fix is reviewed and merged). ## How was this patch tested? I added a new unit test to `TaskSchedulerImplSuite`. You can check out this PR as of 25e455e to see the failing test. cc kayousterhout, markhamstra, rxin for review. Author: Josh Rosen <joshrosen@databricks.com> Closes #15986 from JoshRosen/fix-leak-following-total-executor-loss.
1 parent f158045 commit 9ff03fa

File tree

3 files changed

+120
-35
lines changed

3 files changed

+120
-35
lines changed

core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala

Lines changed: 48 additions & 32 deletions
Original file line numberDiff line numberDiff line change
@@ -88,10 +88,12 @@ private[spark] class TaskSchedulerImpl(
8888
// Incrementing task IDs
8989
val nextTaskId = new AtomicLong(0)
9090

91-
// Number of tasks running on each executor
92-
private val executorIdToTaskCount = new HashMap[String, Int]
91+
// IDs of the tasks running on each executor
92+
private val executorIdToRunningTaskIds = new HashMap[String, HashSet[Long]]
9393

94-
def runningTasksByExecutors(): Map[String, Int] = executorIdToTaskCount.toMap
94+
def runningTasksByExecutors(): Map[String, Int] = {
95+
executorIdToRunningTaskIds.toMap.mapValues(_.size)
96+
}
9597

9698
// The set of executors we have on each host; this is used to compute hostsAlive, which
9799
// in turn is used to decide when we can attain data locality on a given host
@@ -259,7 +261,7 @@ private[spark] class TaskSchedulerImpl(
259261
val tid = task.taskId
260262
taskIdToTaskSetManager(tid) = taskSet
261263
taskIdToExecutorId(tid) = execId
262-
executorIdToTaskCount(execId) += 1
264+
executorIdToRunningTaskIds(execId).add(tid)
263265
executorsByHost(host) += execId
264266
availableCpus(i) -= CPUS_PER_TASK
265267
assert(availableCpus(i) >= 0)
@@ -288,7 +290,7 @@ private[spark] class TaskSchedulerImpl(
288290
var newExecAvail = false
289291
for (o <- offers) {
290292
executorIdToHost(o.executorId) = o.host
291-
executorIdToTaskCount.getOrElseUpdate(o.executorId, 0)
293+
executorIdToRunningTaskIds.getOrElseUpdate(o.executorId, HashSet[Long]())
292294
if (!executorsByHost.contains(o.host)) {
293295
executorsByHost(o.host) = new HashSet[String]()
294296
executorAdded(o.executorId, o.host)
@@ -335,38 +337,34 @@ private[spark] class TaskSchedulerImpl(
335337
var reason: Option[ExecutorLossReason] = None
336338
synchronized {
337339
try {
338-
if (state == TaskState.LOST && taskIdToExecutorId.contains(tid)) {
339-
// We lost this entire executor, so remember that it's gone
340-
val execId = taskIdToExecutorId(tid)
341-
342-
if (executorIdToTaskCount.contains(execId)) {
343-
reason = Some(
344-
SlaveLost(s"Task $tid was lost, so marking the executor as lost as well."))
345-
removeExecutor(execId, reason.get)
346-
failedExecutor = Some(execId)
347-
}
348-
}
349340
taskIdToTaskSetManager.get(tid) match {
350341
case Some(taskSet) =>
351-
if (TaskState.isFinished(state)) {
352-
taskIdToTaskSetManager.remove(tid)
353-
taskIdToExecutorId.remove(tid).foreach { execId =>
354-
if (executorIdToTaskCount.contains(execId)) {
355-
executorIdToTaskCount(execId) -= 1
356-
}
342+
if (state == TaskState.LOST) {
343+
// TaskState.LOST is only used by the deprecated Mesos fine-grained scheduling mode,
344+
// where each executor corresponds to a single task, so mark the executor as failed.
345+
val execId = taskIdToExecutorId.getOrElse(tid, throw new IllegalStateException(
346+
"taskIdToTaskSetManager.contains(tid) <=> taskIdToExecutorId.contains(tid)"))
347+
if (executorIdToRunningTaskIds.contains(execId)) {
348+
reason = Some(
349+
SlaveLost(s"Task $tid was lost, so marking the executor as lost as well."))
350+
removeExecutor(execId, reason.get)
351+
failedExecutor = Some(execId)
357352
}
358353
}
359-
if (state == TaskState.FINISHED) {
360-
taskSet.removeRunningTask(tid)
361-
taskResultGetter.enqueueSuccessfulTask(taskSet, tid, serializedData)
362-
} else if (Set(TaskState.FAILED, TaskState.KILLED, TaskState.LOST).contains(state)) {
354+
if (TaskState.isFinished(state)) {
355+
cleanupTaskState(tid)
363356
taskSet.removeRunningTask(tid)
364-
taskResultGetter.enqueueFailedTask(taskSet, tid, state, serializedData)
357+
if (state == TaskState.FINISHED) {
358+
taskResultGetter.enqueueSuccessfulTask(taskSet, tid, serializedData)
359+
} else if (Set(TaskState.FAILED, TaskState.KILLED, TaskState.LOST).contains(state)) {
360+
taskResultGetter.enqueueFailedTask(taskSet, tid, state, serializedData)
361+
}
365362
}
366363
case None =>
367364
logError(
368365
("Ignoring update with state %s for TID %s because its task set is gone (this is " +
369-
"likely the result of receiving duplicate task finished status updates)")
366+
"likely the result of receiving duplicate task finished status updates) or its " +
367+
"executor has been marked as failed.")
370368
.format(state, tid))
371369
}
372370
} catch {
@@ -477,7 +475,7 @@ private[spark] class TaskSchedulerImpl(
477475
var failedExecutor: Option[String] = None
478476

479477
synchronized {
480-
if (executorIdToTaskCount.contains(executorId)) {
478+
if (executorIdToRunningTaskIds.contains(executorId)) {
481479
val hostPort = executorIdToHost(executorId)
482480
logExecutorLoss(executorId, hostPort, reason)
483481
removeExecutor(executorId, reason)
@@ -519,13 +517,31 @@ private[spark] class TaskSchedulerImpl(
519517
logError(s"Lost executor $executorId on $hostPort: $reason")
520518
}
521519

520+
/**
521+
* Cleans up the TaskScheduler's state for tracking the given task.
522+
*/
523+
private def cleanupTaskState(tid: Long): Unit = {
524+
taskIdToTaskSetManager.remove(tid)
525+
taskIdToExecutorId.remove(tid).foreach { executorId =>
526+
executorIdToRunningTaskIds.get(executorId).foreach { _.remove(tid) }
527+
}
528+
}
529+
522530
/**
523531
* Remove an executor from all our data structures and mark it as lost. If the executor's loss
524532
* reason is not yet known, do not yet remove its association with its host nor update the status
525533
* of any running tasks, since the loss reason defines whether we'll fail those tasks.
526534
*/
527535
private def removeExecutor(executorId: String, reason: ExecutorLossReason) {
528-
executorIdToTaskCount -= executorId
536+
// The tasks on the lost executor may not send any more status updates (because the executor
537+
// has been lost), so they should be cleaned up here.
538+
executorIdToRunningTaskIds.remove(executorId).foreach { taskIds =>
539+
logDebug("Cleaning up TaskScheduler state for tasks " +
540+
s"${taskIds.mkString("[", ",", "]")} on failed executor $executorId")
541+
// We do not notify the TaskSetManager of the task failures because that will
542+
// happen below in the rootPool.executorLost() call.
543+
taskIds.foreach(cleanupTaskState)
544+
}
529545

530546
val host = executorIdToHost(executorId)
531547
val execs = executorsByHost.getOrElse(host, new HashSet)
@@ -563,11 +579,11 @@ private[spark] class TaskSchedulerImpl(
563579
}
564580

565581
def isExecutorAlive(execId: String): Boolean = synchronized {
566-
executorIdToTaskCount.contains(execId)
582+
executorIdToRunningTaskIds.contains(execId)
567583
}
568584

569585
def isExecutorBusy(execId: String): Boolean = synchronized {
570-
executorIdToTaskCount.getOrElse(execId, -1) > 0
586+
executorIdToRunningTaskIds.get(execId).exists(_.nonEmpty)
571587
}
572588

573589
// By default, rack is unknown

core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -433,10 +433,11 @@ class StandaloneDynamicAllocationSuite
433433
assert(executors.size === 2)
434434

435435
// simulate running a task on the executor
436-
val getMap = PrivateMethod[mutable.HashMap[String, Int]]('executorIdToTaskCount)
436+
val getMap =
437+
PrivateMethod[mutable.HashMap[String, mutable.HashSet[Long]]]('executorIdToRunningTaskIds)
437438
val taskScheduler = sc.taskScheduler.asInstanceOf[TaskSchedulerImpl]
438-
val executorIdToTaskCount = taskScheduler invokePrivate getMap()
439-
executorIdToTaskCount(executors.head) = 1
439+
val executorIdToRunningTaskIds = taskScheduler invokePrivate getMap()
440+
executorIdToRunningTaskIds(executors.head) = mutable.HashSet(1L)
440441
// kill the busy executor without force; this should fail
441442
assert(!killExecutor(sc, executors.head, force = false))
442443
apps = getApplications()

core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala

Lines changed: 68 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,8 @@
1717

1818
package org.apache.spark.scheduler
1919

20+
import java.nio.ByteBuffer
21+
2022
import org.apache.spark._
2123
import org.apache.spark.internal.Logging
2224

@@ -274,4 +276,70 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with L
274276
assert("executor1" === taskDescriptions3(0).executorId)
275277
}
276278

279+
test("if an executor is lost then the state for its running tasks is cleaned up (SPARK-18553)") {
280+
sc = new SparkContext("local", "TaskSchedulerImplSuite")
281+
val taskScheduler = new TaskSchedulerImpl(sc)
282+
taskScheduler.initialize(new FakeSchedulerBackend)
283+
// Need to initialize a DAGScheduler for the taskScheduler to use for callbacks.
284+
new DAGScheduler(sc, taskScheduler) {
285+
override def taskStarted(task: Task[_], taskInfo: TaskInfo) {}
286+
override def executorAdded(execId: String, host: String) {}
287+
}
288+
289+
val e0Offers = Seq(WorkerOffer("executor0", "host0", 1))
290+
val attempt1 = FakeTask.createTaskSet(1)
291+
292+
// submit attempt 1, offer resources, task gets scheduled
293+
taskScheduler.submitTasks(attempt1)
294+
val taskDescriptions = taskScheduler.resourceOffers(e0Offers).flatten
295+
assert(1 === taskDescriptions.length)
296+
297+
// mark executor0 as dead
298+
taskScheduler.executorLost("executor0", SlaveLost())
299+
assert(!taskScheduler.isExecutorAlive("executor0"))
300+
assert(!taskScheduler.hasExecutorsAliveOnHost("host0"))
301+
assert(taskScheduler.getExecutorsAliveOnHost("host0").isEmpty)
302+
303+
304+
// Check that state associated with the lost task attempt is cleaned up:
305+
assert(taskScheduler.taskIdToExecutorId.isEmpty)
306+
assert(taskScheduler.taskIdToTaskSetManager.isEmpty)
307+
assert(taskScheduler.runningTasksByExecutors().get("executor0").isEmpty)
308+
}
309+
310+
test("if a task finishes with TaskState.LOST its executor is marked as dead") {
311+
sc = new SparkContext("local", "TaskSchedulerImplSuite")
312+
val taskScheduler = new TaskSchedulerImpl(sc)
313+
taskScheduler.initialize(new FakeSchedulerBackend)
314+
// Need to initialize a DAGScheduler for the taskScheduler to use for callbacks.
315+
new DAGScheduler(sc, taskScheduler) {
316+
override def taskStarted(task: Task[_], taskInfo: TaskInfo) {}
317+
override def executorAdded(execId: String, host: String) {}
318+
}
319+
320+
val e0Offers = Seq(WorkerOffer("executor0", "host0", 1))
321+
val attempt1 = FakeTask.createTaskSet(1)
322+
323+
// submit attempt 1, offer resources, task gets scheduled
324+
taskScheduler.submitTasks(attempt1)
325+
val taskDescriptions = taskScheduler.resourceOffers(e0Offers).flatten
326+
assert(1 === taskDescriptions.length)
327+
328+
// Report the task as failed with TaskState.LOST
329+
taskScheduler.statusUpdate(
330+
tid = taskDescriptions.head.taskId,
331+
state = TaskState.LOST,
332+
serializedData = ByteBuffer.allocate(0)
333+
)
334+
335+
// Check that state associated with the lost task attempt is cleaned up:
336+
assert(taskScheduler.taskIdToExecutorId.isEmpty)
337+
assert(taskScheduler.taskIdToTaskSetManager.isEmpty)
338+
assert(taskScheduler.runningTasksByExecutors().get("executor0").isEmpty)
339+
340+
// Check that the executor has been marked as dead
341+
assert(!taskScheduler.isExecutorAlive("executor0"))
342+
assert(!taskScheduler.hasExecutorsAliveOnHost("host0"))
343+
assert(taskScheduler.getExecutorsAliveOnHost("host0").isEmpty)
344+
}
277345
}

0 commit comments

Comments
 (0)