You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by rx...@apache.org on 2014/11/28 00:54:43 UTC

spark git commit: [SPARK-4626] Kill a task only if the executorId is (still) registered with the scheduler

Repository: spark
Updated Branches:
  refs/heads/master 5d7fe178b -> 84376d313


[SPARK-4626] Kill a task only if the executorId is (still) registered with the scheduler

Author: roxchkplusony <ro...@gmail.com>

Closes #3483 from roxchkplusony/bugfix/4626 and squashes the following commits:

aba9184 [roxchkplusony] replace warning message per review
5e7fdea [roxchkplusony] [SPARK-4626] Kill a task only if the executorId is (still) registered with the scheduler


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/84376d31
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/84376d31
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/84376d31

Branch: refs/heads/master
Commit: 84376d31392858f7df215ddb3f05419181152e68
Parents: 5d7fe17
Author: roxchkplusony <ro...@gmail.com>
Authored: Thu Nov 27 15:54:40 2014 -0800
Committer: Reynold Xin <rx...@databricks.com>
Committed: Thu Nov 27 15:54:40 2014 -0800

----------------------------------------------------------------------
 .../scheduler/cluster/CoarseGrainedSchedulerBackend.scala    | 8 +++++++-
 1 file changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/84376d31/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
index 047fae1..88b196a 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
@@ -127,7 +127,13 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste
         makeOffers()
 
       case KillTask(taskId, executorId, interruptThread) =>
-        executorDataMap(executorId).executorActor ! KillTask(taskId, executorId, interruptThread)
+        executorDataMap.get(executorId) match {
+          case Some(executorInfo) =>
+            executorInfo.executorActor ! KillTask(taskId, executorId, interruptThread)
+          case None =>
+            // Ignoring the task kill since the executor is not registered.
+            logWarning(s"Attempted to kill task $taskId for unknown executor $executorId.")
+        }
 
       case StopDriver =>
         sender ! true


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org