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

git commit: [SPARK-3984] [SPARK-3983] Fix incorrect scheduler delay and display task deserialization time in UI

Repository: spark
Updated Branches:
  refs/heads/master 4c42986cc -> a46497eec


[SPARK-3984] [SPARK-3983] Fix incorrect scheduler delay and display task deserialization time in UI

This commit fixes the scheduler delay in the UI (which previously
included things that are not scheduler delay, like time to
deserialize the task and serialize the result), and also
adds information about time to deserialize tasks to the optional
additional metrics.  Time to deserialize the task can be large relative
to task time for short jobs, and understanding when it is high can help
developers realize that they should try to reduce closure size (e.g, by including
less data in the task description).

cc shivaram etrain

Author: Kay Ousterhout <ka...@gmail.com>

Closes #2832 from kayousterhout/SPARK-3983 and squashes the following commits:

0c1398e [Kay Ousterhout] Fixed ordering
531575d [Kay Ousterhout] Removed executor launch time
1f13afe [Kay Ousterhout] Minor spacing fixes
335be4b [Kay Ousterhout] Made metrics hideable
5bc3cba [Kay Ousterhout] [SPARK-3984] [SPARK-3983] Improve UI task metrics.


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

Branch: refs/heads/master
Commit: a46497eecc50f854c5c5701dc2b8a2468b76c085
Parents: 4c42986
Author: Kay Ousterhout <ka...@gmail.com>
Authored: Wed Nov 5 15:30:31 2014 -0800
Committer: Kay Ousterhout <ka...@gmail.com>
Committed: Wed Nov 5 15:30:31 2014 -0800

----------------------------------------------------------------------
 .../org/apache/spark/executor/Executor.scala    |  4 +--
 .../scala/org/apache/spark/ui/ToolTips.scala    |  3 ++
 .../org/apache/spark/ui/jobs/StagePage.scala    | 31 +++++++++++++++++++-
 .../spark/ui/jobs/TaskDetailsClassNames.scala   |  1 +
 4 files changed, 36 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/a46497ee/core/src/main/scala/org/apache/spark/executor/Executor.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala
index abc1dd0..9611457 100644
--- a/core/src/main/scala/org/apache/spark/executor/Executor.scala
+++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala
@@ -161,7 +161,7 @@ private[spark] class Executor(
     }
 
     override def run() {
-      val startTime = System.currentTimeMillis()
+      val deserializeStartTime = System.currentTimeMillis()
       Thread.currentThread.setContextClassLoader(replClassLoader)
       val ser = SparkEnv.get.closureSerializer.newInstance()
       logInfo(s"Running $taskName (TID $taskId)")
@@ -206,7 +206,7 @@ private[spark] class Executor(
         val afterSerialization = System.currentTimeMillis()
 
         for (m <- task.metrics) {
-          m.executorDeserializeTime = taskStart - startTime
+          m.executorDeserializeTime = taskStart - deserializeStartTime
           m.executorRunTime = taskFinish - taskStart
           m.jvmGCTime = gcTime - startGCTime
           m.resultSerializationTime = afterSerialization - beforeSerialization

http://git-wip-us.apache.org/repos/asf/spark/blob/a46497ee/core/src/main/scala/org/apache/spark/ui/ToolTips.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala
index f02904d..51dc08f 100644
--- a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala
+++ b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala
@@ -24,6 +24,9 @@ private[spark] object ToolTips {
        scheduler delay is large, consider decreasing the size of tasks or decreasing the size
        of task results."""
 
+  val TASK_DESERIALIZATION_TIME =
+    """Time spent deserializating the task closure on the executor."""
+
   val INPUT = "Bytes read from Hadoop or from Spark storage."
 
   val SHUFFLE_WRITE = "Bytes written to disk in order to be read by a shuffle in a future stage."

http://git-wip-us.apache.org/repos/asf/spark/blob/a46497ee/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
index 7cc03b7..63ed5fc 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
@@ -114,6 +114,13 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") {
               </li>
               <li>
                 <span data-toggle="tooltip"
+                      title={ToolTips.TASK_DESERIALIZATION_TIME} data-placement="right">
+                  <input type="checkbox" name={TaskDetailsClassNames.TASK_DESERIALIZATION_TIME}/>
+                  <span class="additional-metric-title">Task Deserialization Time</span>
+                </span>
+              </li>
+              <li>
+                <span data-toggle="tooltip"
                       title={ToolTips.GC_TIME} data-placement="right">
                   <input type="checkbox" name={TaskDetailsClassNames.GC_TIME}/>
                   <span class="additional-metric-title">GC Time</span>
@@ -147,6 +154,7 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") {
           ("Index", ""), ("ID", ""), ("Attempt", ""), ("Status", ""), ("Locality Level", ""),
           ("Executor ID / Host", ""), ("Launch Time", ""), ("Duration", ""),
           ("Scheduler Delay", TaskDetailsClassNames.SCHEDULER_DELAY),
+          ("Task Deserialization Time", TaskDetailsClassNames.TASK_DESERIALIZATION_TIME),
           ("GC Time", TaskDetailsClassNames.GC_TIME),
           ("Result Serialization Time", TaskDetailsClassNames.RESULT_SERIALIZATION_TIME),
           ("Getting Result Time", TaskDetailsClassNames.GETTING_RESULT_TIME)) ++
@@ -179,6 +187,17 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") {
             }
           }
 
+          val deserializationTimes = validTasks.map { case TaskUIData(_, metrics, _) =>
+            metrics.get.executorDeserializeTime.toDouble
+          }
+          val deserializationQuantiles =
+            <td>
+              <span data-toggle="tooltip" title={ToolTips.TASK_DESERIALIZATION_TIME}
+                    data-placement="right">
+                Task Deserialization Time
+              </span>
+            </td> +: getFormattedTimeQuantiles(deserializationTimes)
+
           val serviceTimes = validTasks.map { case TaskUIData(_, metrics, _) =>
             metrics.get.executorRunTime.toDouble
           }
@@ -266,6 +285,9 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") {
           val listings: Seq[Seq[Node]] = Seq(
             <tr>{serviceQuantiles}</tr>,
             <tr class={TaskDetailsClassNames.SCHEDULER_DELAY}>{schedulerDelayQuantiles}</tr>,
+            <tr class={TaskDetailsClassNames.TASK_DESERIALIZATION_TIME}>
+              {deserializationQuantiles}
+            </tr>
             <tr class={TaskDetailsClassNames.GC_TIME}>{gcQuantiles}</tr>,
             <tr class={TaskDetailsClassNames.RESULT_SERIALIZATION_TIME}>
               {serializationQuantiles}
@@ -314,6 +336,7 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") {
         else metrics.map(m => UIUtils.formatDuration(m.executorRunTime)).getOrElse("")
       val schedulerDelay = metrics.map(getSchedulerDelay(info, _)).getOrElse(0L)
       val gcTime = metrics.map(_.jvmGCTime).getOrElse(0L)
+      val taskDeserializationTime = metrics.map(_.executorDeserializeTime).getOrElse(0L)
       val serializationTime = metrics.map(_.resultSerializationTime).getOrElse(0L)
       val gettingResultTime = info.gettingResultTime
 
@@ -367,6 +390,10 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") {
             class={TaskDetailsClassNames.SCHEDULER_DELAY}>
           {UIUtils.formatDuration(schedulerDelay.toLong)}
         </td>
+        <td sorttable_customkey={taskDeserializationTime.toString}
+            class={TaskDetailsClassNames.TASK_DESERIALIZATION_TIME}>
+          {UIUtils.formatDuration(taskDeserializationTime.toLong)}
+        </td>
         <td sorttable_customkey={gcTime.toString} class={TaskDetailsClassNames.GC_TIME}>
           {if (gcTime > 0) UIUtils.formatDuration(gcTime) else ""}
         </td>
@@ -424,6 +451,8 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") {
         (info.finishTime - info.launchTime)
       }
     }
-    totalExecutionTime - metrics.executorRunTime
+    val executorOverhead = (metrics.executorDeserializeTime +
+      metrics.resultSerializationTime)
+    totalExecutionTime - metrics.executorRunTime - executorOverhead
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/a46497ee/core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala b/core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala
index 23d672c..eb371bd 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala
@@ -24,6 +24,7 @@ package org.apache.spark.ui.jobs
 private object TaskDetailsClassNames {
   val SCHEDULER_DELAY = "scheduler_delay"
   val GC_TIME = "gc_time"
+  val TASK_DESERIALIZATION_TIME = "deserialization_time"
   val RESULT_SERIALIZATION_TIME = "serialization_time"
   val GETTING_RESULT_TIME = "getting_result_time"
 }


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