[SPARK-20945] Fix TID key not found in TaskSchedulerImpl
## What changes were proposed in this pull request? This pull request fix the TaskScheulerImpl bug in some condition. Detail see: https://issues.apache.org/jira/browse/SPARK-20945 (Please fill in changes proposed in this fix) ## How was this patch tested? manual tests (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Please review http://spark.apache.org/contributing.html before opening a pull request. Author: liupengcheng <liupengcheng@xiaomi.com> Author: PengchengLiu <pengchengliu_bupt@163.com> Closes #18171 from liupc/Fix-tid-key-not-found-in-TaskSchedulerImpl.
This commit is contained in:
parent
dec9aa3b37
commit
2d39711b05
|
@ -240,8 +240,8 @@ private[spark] class TaskSchedulerImpl private[scheduler](
|
|||
// 2. The task set manager has been created but no tasks has been scheduled. In this case,
|
||||
// simply abort the stage.
|
||||
tsm.runningTasksSet.foreach { tid =>
|
||||
val execId = taskIdToExecutorId(tid)
|
||||
backend.killTask(tid, execId, interruptThread, reason = "stage cancelled")
|
||||
taskIdToExecutorId.get(tid).foreach(execId =>
|
||||
backend.killTask(tid, execId, interruptThread, reason = "Stage cancelled"))
|
||||
}
|
||||
tsm.abort("Stage %s cancelled".format(stageId))
|
||||
logInfo("Stage %d was cancelled".format(stageId))
|
||||
|
|
Loading…
Reference in a new issue