diff --git a/core/src/main/scala/spark/executor/Executor.scala b/core/src/main/scala/spark/executor/Executor.scala index 82e5f5a0155aeb953b4bdaae402ad20c1b06d6be..2e81151882837e92b4e97131b6c9e5d6ab5661e1 100644 --- a/core/src/main/scala/spark/executor/Executor.scala +++ b/core/src/main/scala/spark/executor/Executor.scala @@ -126,7 +126,6 @@ private[spark] class Executor(executorId: String, slaveHostname: String, propert val taskFinish = System.currentTimeMillis() task.metrics.foreach{ m => m.hostname = Utils.localHostName - m.executorId = executorId m.executorDeserializeTime = (taskStart - startTime).toInt m.executorRunTime = (taskFinish - taskStart).toInt } diff --git a/core/src/main/scala/spark/executor/TaskMetrics.scala b/core/src/main/scala/spark/executor/TaskMetrics.scala index e14dc57501c5fc054d04770955ca2bb35b392d69..315162783981f8e2d18e61f86f90c529c4923492 100644 --- a/core/src/main/scala/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/spark/executor/TaskMetrics.scala @@ -23,11 +23,6 @@ class TaskMetrics extends Serializable { */ var hostname: String = _ - /** - * Executor's ID the task runs on - */ - var executorId: String = _ - /** * Time taken on the executor to deserialize this task */ diff --git a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala index a981c680d2201f0a2e8956e4a2ae883a30a6b0ae..40bee325b2abfd4282c1770e97f561591058bbeb 100644 --- a/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/spark/ui/exec/ExecutorsUI.scala @@ -111,16 +111,14 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { HashMap[String, ArrayBuffer[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]]() override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { - val eid = taskEnd.taskMetrics.executorId + val eid = taskEnd.taskInfo.executorId val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { case e: ExceptionFailure => executorToTasksFailed(eid) = executorToTasksFailed.getOrElse(eid, 0) + 1 - logInfo("Executor %s has %s failed tasks.".format(eid, executorToTasksFailed(eid))) (Some(e), e.metrics) case _ => executorToTasksComplete(eid) = executorToTasksComplete.getOrElse(eid, 0) + 1 - logInfo("Executor %s has %s completed tasks.".format(eid, executorToTasksComplete(eid))) (None, Some(taskEnd.taskMetrics)) } val taskList = executorToTaskInfos.getOrElse(