@@ -93,10 +93,12 @@ private[spark] class TaskSchedulerImpl(
9393 // Incrementing task IDs
9494 val nextTaskId = new AtomicLong (0 )
9595
96- // Number of tasks running on each executor
97- private val executorIdToTaskCount = new HashMap [String , Int ]
96+ // IDs of the tasks running on each executor
97+ private val executorIdToRunningTaskIds = new HashMap [String , HashSet [ Long ] ]
9898
99- def runningTasksByExecutors (): Map [String , Int ] = executorIdToTaskCount.toMap
99+ def runningTasksByExecutors (): Map [String , Int ] = {
100+ executorIdToRunningTaskIds.toMap.mapValues(_.size)
101+ }
100102
101103 // The set of executors we have on each host; this is used to compute hostsAlive, which
102104 // in turn is used to decide when we can attain data locality on a given host
@@ -264,7 +266,7 @@ private[spark] class TaskSchedulerImpl(
264266 val tid = task.taskId
265267 taskIdToTaskSetManager(tid) = taskSet
266268 taskIdToExecutorId(tid) = execId
267- executorIdToTaskCount (execId) += 1
269+ executorIdToRunningTaskIds (execId).add(tid)
268270 availableCpus(i) -= CPUS_PER_TASK
269271 assert(availableCpus(i) >= 0 )
270272 launchedTask = true
@@ -294,11 +296,11 @@ private[spark] class TaskSchedulerImpl(
294296 if (! hostToExecutors.contains(o.host)) {
295297 hostToExecutors(o.host) = new HashSet [String ]()
296298 }
297- if (! executorIdToTaskCount .contains(o.executorId)) {
299+ if (! executorIdToRunningTaskIds .contains(o.executorId)) {
298300 hostToExecutors(o.host) += o.executorId
299301 executorAdded(o.executorId, o.host)
300302 executorIdToHost(o.executorId) = o.host
301- executorIdToTaskCount (o.executorId) = 0
303+ executorIdToRunningTaskIds (o.executorId) = HashSet [ Long ]()
302304 newExecAvail = true
303305 }
304306 for (rack <- getRackForHost(o.host)) {
@@ -349,38 +351,34 @@ private[spark] class TaskSchedulerImpl(
349351 var reason : Option [ExecutorLossReason ] = None
350352 synchronized {
351353 try {
352- if (state == TaskState .LOST && taskIdToExecutorId.contains(tid)) {
353- // We lost this entire executor, so remember that it's gone
354- val execId = taskIdToExecutorId(tid)
355-
356- if (executorIdToTaskCount.contains(execId)) {
357- reason = Some (
358- SlaveLost (s " Task $tid was lost, so marking the executor as lost as well. " ))
359- removeExecutor(execId, reason.get)
360- failedExecutor = Some (execId)
361- }
362- }
363354 taskIdToTaskSetManager.get(tid) match {
364355 case Some (taskSet) =>
365- if (TaskState .isFinished(state)) {
366- taskIdToTaskSetManager.remove(tid)
367- taskIdToExecutorId.remove(tid).foreach { execId =>
368- if (executorIdToTaskCount.contains(execId)) {
369- executorIdToTaskCount(execId) -= 1
370- }
356+ if (state == TaskState .LOST ) {
357+ // TaskState.LOST is only used by the deprecated Mesos fine-grained scheduling mode,
358+ // where each executor corresponds to a single task, so mark the executor as failed.
359+ val execId = taskIdToExecutorId.getOrElse(tid, throw new IllegalStateException (
360+ " taskIdToTaskSetManager.contains(tid) <=> taskIdToExecutorId.contains(tid)" ))
361+ if (executorIdToRunningTaskIds.contains(execId)) {
362+ reason = Some (
363+ SlaveLost (s " Task $tid was lost, so marking the executor as lost as well. " ))
364+ removeExecutor(execId, reason.get)
365+ failedExecutor = Some (execId)
371366 }
372367 }
373- if (state == TaskState .FINISHED ) {
374- taskSet.removeRunningTask(tid)
375- taskResultGetter.enqueueSuccessfulTask(taskSet, tid, serializedData)
376- } else if (Set (TaskState .FAILED , TaskState .KILLED , TaskState .LOST ).contains(state)) {
368+ if (TaskState .isFinished(state)) {
369+ cleanupTaskState(tid)
377370 taskSet.removeRunningTask(tid)
378- taskResultGetter.enqueueFailedTask(taskSet, tid, state, serializedData)
371+ if (state == TaskState .FINISHED ) {
372+ taskResultGetter.enqueueSuccessfulTask(taskSet, tid, serializedData)
373+ } else if (Set (TaskState .FAILED , TaskState .KILLED , TaskState .LOST ).contains(state)) {
374+ taskResultGetter.enqueueFailedTask(taskSet, tid, state, serializedData)
375+ }
379376 }
380377 case None =>
381378 logError(
382379 (" Ignoring update with state %s for TID %s because its task set is gone (this is " +
383- " likely the result of receiving duplicate task finished status updates)" )
380+ " likely the result of receiving duplicate task finished status updates) or its " +
381+ " executor has been marked as failed." )
384382 .format(state, tid))
385383 }
386384 } catch {
@@ -491,7 +489,7 @@ private[spark] class TaskSchedulerImpl(
491489 var failedExecutor : Option [String ] = None
492490
493491 synchronized {
494- if (executorIdToTaskCount .contains(executorId)) {
492+ if (executorIdToRunningTaskIds .contains(executorId)) {
495493 val hostPort = executorIdToHost(executorId)
496494 logExecutorLoss(executorId, hostPort, reason)
497495 removeExecutor(executorId, reason)
@@ -533,13 +531,31 @@ private[spark] class TaskSchedulerImpl(
533531 logError(s " Lost executor $executorId on $hostPort: $reason" )
534532 }
535533
534+ /**
535+ * Cleans up the TaskScheduler's state for tracking the given task.
536+ */
537+ private def cleanupTaskState (tid : Long ): Unit = {
538+ taskIdToTaskSetManager.remove(tid)
539+ taskIdToExecutorId.remove(tid).foreach { executorId =>
540+ executorIdToRunningTaskIds.get(executorId).foreach { _.remove(tid) }
541+ }
542+ }
543+
536544 /**
537545 * Remove an executor from all our data structures and mark it as lost. If the executor's loss
538546 * reason is not yet known, do not yet remove its association with its host nor update the status
539547 * of any running tasks, since the loss reason defines whether we'll fail those tasks.
540548 */
541549 private def removeExecutor (executorId : String , reason : ExecutorLossReason ) {
542- executorIdToTaskCount -= executorId
550+ // The tasks on the lost executor may not send any more status updates (because the executor
551+ // has been lost), so they should be cleaned up here.
552+ executorIdToRunningTaskIds.remove(executorId).foreach { taskIds =>
553+ logDebug(" Cleaning up TaskScheduler state for tasks " +
554+ s " ${taskIds.mkString(" [" , " ," , " ]" )} on failed executor $executorId" )
555+ // We do not notify the TaskSetManager of the task failures because that will
556+ // happen below in the rootPool.executorLost() call.
557+ taskIds.foreach(cleanupTaskState)
558+ }
543559
544560 val host = executorIdToHost(executorId)
545561 val execs = hostToExecutors.getOrElse(host, new HashSet )
@@ -577,11 +593,11 @@ private[spark] class TaskSchedulerImpl(
577593 }
578594
579595 def isExecutorAlive (execId : String ): Boolean = synchronized {
580- executorIdToTaskCount .contains(execId)
596+ executorIdToRunningTaskIds .contains(execId)
581597 }
582598
583599 def isExecutorBusy (execId : String ): Boolean = synchronized {
584- executorIdToTaskCount.getOrElse (execId, - 1 ) > 0
600+ executorIdToRunningTaskIds.get (execId).exists(_.nonEmpty)
585601 }
586602
587603 // By default, rack is unknown
0 commit comments