This is an automated email from the ASF dual-hosted git repository.

hvanhovell pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new 00a8c85  [SPARK-27071][CORE] Expose additional metrics in 
status.api.v1.StageData
00a8c85 is described below

commit 00a8c85a442402ebfd1bed0e381a2a42d713856a
Author: Tom van Bussel <[email protected]>
AuthorDate: Mon May 27 17:36:22 2019 +0200

    [SPARK-27071][CORE] Expose additional metrics in status.api.v1.StageData
    
    ## What changes were proposed in this pull request?
    
    This PR exposes additional metrics in `status.api.v1.StageData`. These 
metrics were already computed for `LiveStage`, but they were never exposed to 
the user. This includes extra metrics about the JVM GC, executor 
(de)serialization times, shuffle reads and writes, and more.
    
    ## How was this patch tested?
    
    Existing tests.
    
    cc hvanhovell
    
    Closes #24011 from tomvanbussel/SPARK-27071.
    
    Authored-by: Tom van Bussel <[email protected]>
    Signed-off-by: herman <[email protected]>
---
 .../org/apache/spark/status/AppStatusStore.scala   | 81 +++++++++++--------
 .../scala/org/apache/spark/status/LiveEntity.scala | 92 +++++++++++++---------
 .../scala/org/apache/spark/status/api/v1/api.scala | 21 ++++-
 .../scala/org/apache/spark/ui/jobs/JobPage.scala   | 65 +++++++++++----
 .../blacklisting_for_stage_expectation.json        | 21 ++++-
 .../blacklisting_node_for_stage_expectation.json   | 23 ++++--
 .../complete_stage_list_json_expectation.json      | 63 ++++++++++++---
 .../failed_stage_list_json_expectation.json        | 21 ++++-
 .../one_stage_attempt_json_expectation.json        | 21 ++++-
 .../one_stage_json_expectation.json                | 21 ++++-
 .../stage_list_json_expectation.json               | 84 ++++++++++++++++----
 ...age_list_with_accumulable_json_expectation.json | 21 ++++-
 .../stage_with_accumulable_json_expectation.json   | 21 ++++-
 .../scala/org/apache/spark/ui/StagePageSuite.scala | 21 ++++-
 14 files changed, 424 insertions(+), 152 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala 
b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala
index 0487f2f..6240256 100644
--- a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala
+++ b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala
@@ -462,40 +462,53 @@ private[spark] class AppStatusStore(
       .toMap
 
     new v1.StageData(
-      stage.status,
-      stage.stageId,
-      stage.attemptId,
-      stage.numTasks,
-      stage.numActiveTasks,
-      stage.numCompleteTasks,
-      stage.numFailedTasks,
-      stage.numKilledTasks,
-      stage.numCompletedIndices,
-      stage.executorRunTime,
-      stage.executorCpuTime,
-      stage.submissionTime,
-      stage.firstTaskLaunchedTime,
-      stage.completionTime,
-      stage.failureReason,
-      stage.inputBytes,
-      stage.inputRecords,
-      stage.outputBytes,
-      stage.outputRecords,
-      stage.shuffleReadBytes,
-      stage.shuffleReadRecords,
-      stage.shuffleWriteBytes,
-      stage.shuffleWriteRecords,
-      stage.memoryBytesSpilled,
-      stage.diskBytesSpilled,
-      stage.name,
-      stage.description,
-      stage.details,
-      stage.schedulingPool,
-      stage.rddIds,
-      stage.accumulatorUpdates,
-      Some(tasks),
-      Some(executorSummary(stage.stageId, stage.attemptId)),
-      stage.killedTasksSummary)
+      status = stage.status,
+      stageId = stage.stageId,
+      attemptId = stage.attemptId,
+      numTasks = stage.numTasks,
+      numActiveTasks = stage.numActiveTasks,
+      numCompleteTasks = stage.numCompleteTasks,
+      numFailedTasks = stage.numFailedTasks,
+      numKilledTasks = stage.numKilledTasks,
+      numCompletedIndices = stage.numCompletedIndices,
+      submissionTime = stage.submissionTime,
+      firstTaskLaunchedTime = stage.firstTaskLaunchedTime,
+      completionTime = stage.completionTime,
+      failureReason = stage.failureReason,
+      executorDeserializeTime = stage.executorDeserializeTime,
+      executorDeserializeCpuTime = stage.executorDeserializeCpuTime,
+      executorRunTime = stage.executorRunTime,
+      executorCpuTime = stage.executorCpuTime,
+      resultSize = stage.resultSize,
+      jvmGcTime = stage.jvmGcTime,
+      resultSerializationTime = stage.resultSerializationTime,
+      memoryBytesSpilled = stage.memoryBytesSpilled,
+      diskBytesSpilled = stage.diskBytesSpilled,
+      peakExecutionMemory = stage.peakExecutionMemory,
+      inputBytes = stage.inputBytes,
+      inputRecords = stage.inputRecords,
+      outputBytes = stage.outputBytes,
+      outputRecords = stage.outputRecords,
+      shuffleRemoteBlocksFetched = stage.shuffleRemoteBlocksFetched,
+      shuffleLocalBlocksFetched = stage.shuffleLocalBlocksFetched,
+      shuffleFetchWaitTime = stage.shuffleFetchWaitTime,
+      shuffleRemoteBytesRead = stage.shuffleRemoteBytesRead,
+      shuffleRemoteBytesReadToDisk = stage.shuffleRemoteBytesReadToDisk,
+      shuffleLocalBytesRead = stage.shuffleLocalBytesRead,
+      shuffleReadBytes = stage.shuffleReadBytes,
+      shuffleReadRecords = stage.shuffleReadRecords,
+      shuffleWriteBytes = stage.shuffleWriteBytes,
+      shuffleWriteTime = stage.shuffleWriteTime,
+      shuffleWriteRecords = stage.shuffleWriteRecords,
+      name = stage.name,
+      description = stage.description,
+      details = stage.details,
+      schedulingPool = stage.schedulingPool,
+      rddIds = stage.rddIds,
+      accumulatorUpdates = stage.accumulatorUpdates,
+      tasks = Some(tasks),
+      executorSummary = Some(executorSummary(stage.stageId, stage.attemptId)),
+      killedTasksSummary = stage.killedTasksSummary)
   }
 
   def rdd(rddId: Int): v1.RDDStorageInfo = {
diff --git a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala 
b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala
index 6d7b34a..c5a233f 100644
--- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala
+++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala
@@ -394,45 +394,59 @@ private class LiveStage extends LiveEntity {
 
   def toApi(): v1.StageData = {
     new v1.StageData(
-      status,
-      info.stageId,
-      info.attemptNumber,
-
-      info.numTasks,
-      activeTasks,
-      completedTasks,
-      failedTasks,
-      killedTasks,
-      completedIndices.size,
-
-      metrics.executorRunTime,
-      metrics.executorCpuTime,
-      info.submissionTime.map(new Date(_)),
-      if (firstLaunchTime < Long.MaxValue) Some(new Date(firstLaunchTime)) 
else None,
-      info.completionTime.map(new Date(_)),
-      info.failureReason,
-
-      metrics.inputMetrics.bytesRead,
-      metrics.inputMetrics.recordsRead,
-      metrics.outputMetrics.bytesWritten,
-      metrics.outputMetrics.recordsWritten,
-      metrics.shuffleReadMetrics.localBytesRead + 
metrics.shuffleReadMetrics.remoteBytesRead,
-      metrics.shuffleReadMetrics.recordsRead,
-      metrics.shuffleWriteMetrics.bytesWritten,
-      metrics.shuffleWriteMetrics.recordsWritten,
-      metrics.memoryBytesSpilled,
-      metrics.diskBytesSpilled,
-
-      info.name,
-      description,
-      info.details,
-      schedulingPool,
-
-      info.rddInfos.map(_.id),
-      newAccumulatorInfos(info.accumulables.values),
-      None,
-      None,
-      killedSummary)
+      status = status,
+      stageId = info.stageId,
+      attemptId = info.attemptNumber,
+      numTasks = info.numTasks,
+      numActiveTasks = activeTasks,
+      numCompleteTasks = completedTasks,
+      numFailedTasks = failedTasks,
+      numKilledTasks = killedTasks,
+      numCompletedIndices = completedIndices.size,
+
+      submissionTime = info.submissionTime.map(new Date(_)),
+      firstTaskLaunchedTime =
+        if (firstLaunchTime < Long.MaxValue) Some(new Date(firstLaunchTime)) 
else None,
+      completionTime = info.completionTime.map(new Date(_)),
+      failureReason = info.failureReason,
+
+      executorDeserializeTime = metrics.executorDeserializeTime,
+      executorDeserializeCpuTime = metrics.executorDeserializeCpuTime,
+      executorRunTime = metrics.executorRunTime,
+      executorCpuTime = metrics.executorCpuTime,
+      resultSize = metrics.resultSize,
+      jvmGcTime = metrics.jvmGcTime,
+      resultSerializationTime = metrics.resultSerializationTime,
+      memoryBytesSpilled = metrics.memoryBytesSpilled,
+      diskBytesSpilled = metrics.diskBytesSpilled,
+      peakExecutionMemory = metrics.peakExecutionMemory,
+      inputBytes = metrics.inputMetrics.bytesRead,
+      inputRecords = metrics.inputMetrics.recordsRead,
+      outputBytes = metrics.outputMetrics.bytesWritten,
+      outputRecords = metrics.outputMetrics.recordsWritten,
+      shuffleRemoteBlocksFetched = 
metrics.shuffleReadMetrics.remoteBlocksFetched,
+      shuffleLocalBlocksFetched = 
metrics.shuffleReadMetrics.localBlocksFetched,
+      shuffleFetchWaitTime = metrics.shuffleReadMetrics.fetchWaitTime,
+      shuffleRemoteBytesRead = metrics.shuffleReadMetrics.remoteBytesRead,
+      shuffleRemoteBytesReadToDisk = 
metrics.shuffleReadMetrics.remoteBytesReadToDisk,
+      shuffleLocalBytesRead = metrics.shuffleReadMetrics.localBytesRead,
+      shuffleReadBytes =
+        metrics.shuffleReadMetrics.localBytesRead + 
metrics.shuffleReadMetrics.remoteBytesRead,
+      shuffleReadRecords = metrics.shuffleReadMetrics.recordsRead,
+      shuffleWriteBytes = metrics.shuffleWriteMetrics.bytesWritten,
+      shuffleWriteTime = metrics.shuffleWriteMetrics.writeTime,
+      shuffleWriteRecords = metrics.shuffleWriteMetrics.recordsWritten,
+
+      name = info.name,
+      description = description,
+      details = info.details,
+      schedulingPool = schedulingPool,
+
+      rddIds = info.rddInfos.map(_.id),
+      accumulatorUpdates = newAccumulatorInfos(info.accumulables.values),
+      tasks = None,
+      executorSummary = None,
+      killedTasksSummary = killedSummary)
   }
 
   override protected def doUpdate(): Any = {
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala 
b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala
index 60b8bf8..7b3636f 100644
--- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala
@@ -210,23 +210,36 @@ class StageData private[spark](
     val numKilledTasks: Int,
     val numCompletedIndices: Int,
 
-    val executorRunTime: Long,
-    val executorCpuTime: Long,
     val submissionTime: Option[Date],
     val firstTaskLaunchedTime: Option[Date],
     val completionTime: Option[Date],
     val failureReason: Option[String],
 
+    val executorDeserializeTime: Long,
+    val executorDeserializeCpuTime: Long,
+    val executorRunTime: Long,
+    val executorCpuTime: Long,
+    val resultSize: Long,
+    val jvmGcTime: Long,
+    val resultSerializationTime: Long,
+    val memoryBytesSpilled: Long,
+    val diskBytesSpilled: Long,
+    val peakExecutionMemory: Long,
     val inputBytes: Long,
     val inputRecords: Long,
     val outputBytes: Long,
     val outputRecords: Long,
+    val shuffleRemoteBlocksFetched: Long,
+    val shuffleLocalBlocksFetched: Long,
+    val shuffleFetchWaitTime: Long,
+    val shuffleRemoteBytesRead: Long,
+    val shuffleRemoteBytesReadToDisk: Long,
+    val shuffleLocalBytesRead: Long,
     val shuffleReadBytes: Long,
     val shuffleReadRecords: Long,
     val shuffleWriteBytes: Long,
+    val shuffleWriteTime: Long,
     val shuffleWriteRecords: Long,
-    val memoryBytesSpilled: Long,
-    val diskBytesSpilled: Long,
 
     val name: String,
     val description: Option[String],
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala 
b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala
index 46295e7..6e373cf 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala
@@ -203,20 +203,57 @@ private[ui] class JobPage(parent: JobsTab, store: 
AppStatusStore) extends WebUIP
       // stage or if the stage information has been garbage collected
       store.asOption(store.lastStageAttempt(stageId)).getOrElse {
         new v1.StageData(
-          v1.StageStatus.PENDING,
-          stageId,
-          0, 0, 0, 0, 0, 0, 0,
-          0L, 0L, None, None, None, None,
-          0L, 0L, 0L, 0L, 0L, 0L, 0L, 0L, 0L, 0L,
-          "Unknown",
-          None,
-          "Unknown",
-          null,
-          Nil,
-          Nil,
-          None,
-          None,
-          Map())
+          status = v1.StageStatus.PENDING,
+          stageId = stageId,
+          attemptId = 0,
+          numTasks = 0,
+          numActiveTasks = 0,
+          numCompleteTasks = 0,
+          numFailedTasks = 0,
+          numKilledTasks = 0,
+          numCompletedIndices = 0,
+
+          submissionTime = None,
+          firstTaskLaunchedTime = None,
+          completionTime = None,
+          failureReason = None,
+
+          executorDeserializeTime = 0L,
+          executorDeserializeCpuTime = 0L,
+          executorRunTime = 0L,
+          executorCpuTime = 0L,
+          resultSize = 0L,
+          jvmGcTime = 0L,
+          resultSerializationTime = 0L,
+          memoryBytesSpilled = 0L,
+          diskBytesSpilled = 0L,
+          peakExecutionMemory = 0L,
+          inputBytes = 0L,
+          inputRecords = 0L,
+          outputBytes = 0L,
+          outputRecords = 0L,
+          shuffleRemoteBlocksFetched = 0L,
+          shuffleLocalBlocksFetched = 0L,
+          shuffleFetchWaitTime = 0L,
+          shuffleRemoteBytesRead = 0L,
+          shuffleRemoteBytesReadToDisk = 0L,
+          shuffleLocalBytesRead = 0L,
+          shuffleReadBytes = 0L,
+          shuffleReadRecords = 0L,
+          shuffleWriteBytes = 0L,
+          shuffleWriteTime = 0L,
+          shuffleWriteRecords = 0L,
+
+          name = "Unknown",
+          description = None,
+          details = "Unknown",
+          schedulingPool = null,
+
+          rddIds = Nil,
+          accumulatorUpdates = Nil,
+          tasks = None,
+          executorSummary = None,
+          killedTasksSummary = Map())
       }
     }
 
diff --git 
a/core/src/test/resources/HistoryServerExpectations/blacklisting_for_stage_expectation.json
 
b/core/src/test/resources/HistoryServerExpectations/blacklisting_for_stage_expectation.json
index 62e5c12..b18b19f 100644
--- 
a/core/src/test/resources/HistoryServerExpectations/blacklisting_for_stage_expectation.json
+++ 
b/core/src/test/resources/HistoryServerExpectations/blacklisting_for_stage_expectation.json
@@ -8,21 +8,34 @@
   "numFailedTasks" : 2,
   "numKilledTasks" : 0,
   "numCompletedIndices" : 10,
-  "executorRunTime" : 761,
-  "executorCpuTime" : 269916000,
   "submissionTime" : "2018-01-09T10:21:18.152GMT",
   "firstTaskLaunchedTime" : "2018-01-09T10:21:18.347GMT",
   "completionTime" : "2018-01-09T10:21:19.062GMT",
+  "executorDeserializeTime" : 327,
+  "executorDeserializeCpuTime" : 225900000,
+  "executorRunTime" : 761,
+  "executorCpuTime" : 269916000,
+  "resultSize" : 10376,
+  "jvmGcTime" : 27,
+  "resultSerializationTime" : 1,
+  "memoryBytesSpilled" : 0,
+  "diskBytesSpilled" : 0,
+  "peakExecutionMemory" : 0,
   "inputBytes" : 0,
   "inputRecords" : 0,
   "outputBytes" : 0,
   "outputRecords" : 0,
+  "shuffleRemoteBlocksFetched" : 0,
+  "shuffleLocalBlocksFetched" : 0,
+  "shuffleFetchWaitTime" : 0,
+  "shuffleRemoteBytesRead" : 0,
+  "shuffleRemoteBytesReadToDisk" : 0,
+  "shuffleLocalBytesRead" : 0,
   "shuffleReadBytes" : 0,
   "shuffleReadRecords" : 0,
   "shuffleWriteBytes" : 460,
+  "shuffleWriteTime" : 8711515,
   "shuffleWriteRecords" : 10,
-  "memoryBytesSpilled" : 0,
-  "diskBytesSpilled" : 0,
   "name" : "map at <console>:26",
   "details" : 
"org.apache.spark.rdd.RDD.map(RDD.scala:370)\n$line17.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:26)\n$line17.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:34)\n$line17.$read$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:36)\n$line17.$read$$iw$$iw$$iw$$iw$$iw.<init>(<console>:38)\n$line17.$read$$iw$$iw$$iw$$iw.<init>(<console>:40)\n$line17.$read$$iw$$iw$$iw.<init>(<console>:42)\n$line17.$read$$iw$$iw.<init>(<console>:44)\n$line17.$read$$iw.<init>(<console>:46)\n
 [...]
   "schedulingPool" : "default",
diff --git 
a/core/src/test/resources/HistoryServerExpectations/blacklisting_node_for_stage_expectation.json
 
b/core/src/test/resources/HistoryServerExpectations/blacklisting_node_for_stage_expectation.json
index 6e46c88..8d11081 100644
--- 
a/core/src/test/resources/HistoryServerExpectations/blacklisting_node_for_stage_expectation.json
+++ 
b/core/src/test/resources/HistoryServerExpectations/blacklisting_node_for_stage_expectation.json
@@ -8,21 +8,34 @@
   "numFailedTasks" : 4,
   "numKilledTasks" : 0,
   "numCompletedIndices" : 10,
-  "executorRunTime" : 5080,
-  "executorCpuTime" : 1163210819,
   "submissionTime" : "2018-01-18T18:33:12.658GMT",
   "firstTaskLaunchedTime" : "2018-01-18T18:33:12.816GMT",
   "completionTime" : "2018-01-18T18:33:15.279GMT",
+  "executorDeserializeTime" : 3679,
+  "executorDeserializeCpuTime" : 1029819716,
+  "executorRunTime" : 5080,
+  "executorCpuTime" : 1163210819,
+  "resultSize" : 10824,
+  "jvmGcTime" : 370,
+  "resultSerializationTime" : 5,
+  "memoryBytesSpilled" : 0,
+  "diskBytesSpilled" : 0,
+  "peakExecutionMemory" : 0,
   "inputBytes" : 0,
   "inputRecords" : 0,
   "outputBytes" : 0,
   "outputRecords" : 0,
+  "shuffleRemoteBlocksFetched" : 0,
+  "shuffleLocalBlocksFetched" : 0,
+  "shuffleFetchWaitTime" : 0,
+  "shuffleRemoteBytesRead" : 0,
+  "shuffleRemoteBytesReadToDisk" : 0,
+  "shuffleLocalBytesRead" : 0,
   "shuffleReadBytes" : 0,
   "shuffleReadRecords" : 0,
   "shuffleWriteBytes" : 1461,
+  "shuffleWriteTime" : 33251697,
   "shuffleWriteRecords" : 30,
-  "memoryBytesSpilled" : 0,
-  "diskBytesSpilled" : 0,
   "name" : "map at <console>:27",
   "details" : 
"org.apache.spark.rdd.RDD.map(RDD.scala:370)\n$line15.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:27)\n$line15.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:35)\n$line15.$read$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:37)\n$line15.$read$$iw$$iw$$iw$$iw$$iw.<init>(<console>:39)\n$line15.$read$$iw$$iw$$iw$$iw.<init>(<console>:41)\n$line15.$read$$iw$$iw$$iw.<init>(<console>:43)\n$line15.$read$$iw$$iw.<init>(<console>:45)\n$line15.$read$$iw.<init>(<console>:47)\n
 [...]
   "schedulingPool" : "default",
@@ -864,4 +877,4 @@
     }
   },
   "killedTasksSummary" : { }
-}
\ No newline at end of file
+}
diff --git 
a/core/src/test/resources/HistoryServerExpectations/complete_stage_list_json_expectation.json
 
b/core/src/test/resources/HistoryServerExpectations/complete_stage_list_json_expectation.json
index 37b7d72..a47cd26 100644
--- 
a/core/src/test/resources/HistoryServerExpectations/complete_stage_list_json_expectation.json
+++ 
b/core/src/test/resources/HistoryServerExpectations/complete_stage_list_json_expectation.json
@@ -8,21 +8,34 @@
   "numFailedTasks" : 0,
   "numKilledTasks" : 0,
   "numCompletedIndices" : 8,
-  "executorRunTime" : 162,
-  "executorCpuTime" : 0,
   "submissionTime" : "2015-02-03T16:43:07.191GMT",
   "firstTaskLaunchedTime" : "2015-02-03T16:43:07.191GMT",
   "completionTime" : "2015-02-03T16:43:07.226GMT",
+  "executorDeserializeTime" : 36,
+  "executorDeserializeCpuTime" : 0,
+  "executorRunTime" : 162,
+  "executorCpuTime" : 0,
+  "resultSize" : 14496,
+  "jvmGcTime" : 0,
+  "resultSerializationTime" : 1,
+  "memoryBytesSpilled" : 0,
+  "diskBytesSpilled" : 0,
+  "peakExecutionMemory" : 0,
   "inputBytes" : 160,
   "inputRecords" : 0,
   "outputBytes" : 0,
   "outputRecords" : 0,
+  "shuffleRemoteBlocksFetched" : 0,
+  "shuffleLocalBlocksFetched" : 0,
+  "shuffleFetchWaitTime" : 0,
+  "shuffleRemoteBytesRead" : 0,
+  "shuffleRemoteBytesReadToDisk" : 0,
+  "shuffleLocalBytesRead" : 0,
   "shuffleReadBytes" : 0,
   "shuffleReadRecords" : 0,
   "shuffleWriteBytes" : 0,
+  "shuffleWriteTime" : 0,
   "shuffleWriteRecords" : 0,
-  "memoryBytesSpilled" : 0,
-  "diskBytesSpilled" : 0,
   "name" : "count at <console>:17",
   "details" : 
"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line19.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:17)\n$line19.$read$$iwC$$iwC$$iwC.<init>(<console>:22)\n$line19.$read$$iwC$$iwC.<init>(<console>:24)\n$line19.$read$$iwC.<init>(<console>:26)\n$line19.$read.<init>(<console>:28)\n$line19.$read$.<init>(<console>:32)\n$line19.$read$.<clinit>(<console>)\n$line19.$eval$.<init>(<console>:7)\n$line19.$eval$.<clinit>(<console>)\n$line19.$eval.$print(<console>)\nsun.reflect.NativeMeth
 [...]
   "schedulingPool" : "default",
@@ -39,21 +52,34 @@
   "numFailedTasks" : 0,
   "numKilledTasks" : 0,
   "numCompletedIndices" : 8,
-  "executorRunTime" : 3476,
-  "executorCpuTime" : 0,
   "submissionTime" : "2015-02-03T16:43:05.829GMT",
   "firstTaskLaunchedTime" : "2015-02-03T16:43:05.829GMT",
   "completionTime" : "2015-02-03T16:43:06.286GMT",
+  "executorDeserializeTime" : 13,
+  "executorDeserializeCpuTime" : 0,
+  "executorRunTime" : 3476,
+  "executorCpuTime" : 0,
+  "resultSize" : 15216,
+  "jvmGcTime" : 152,
+  "resultSerializationTime" : 9,
+  "memoryBytesSpilled" : 0,
+  "diskBytesSpilled" : 0,
+  "peakExecutionMemory" : 0,
   "inputBytes" : 28000128,
   "inputRecords" : 0,
   "outputBytes" : 0,
   "outputRecords" : 0,
+  "shuffleRemoteBlocksFetched" : 0,
+  "shuffleLocalBlocksFetched" : 0,
+  "shuffleFetchWaitTime" : 0,
+  "shuffleRemoteBytesRead" : 0,
+  "shuffleRemoteBytesReadToDisk" : 0,
+  "shuffleLocalBytesRead" : 0,
   "shuffleReadBytes" : 0,
   "shuffleReadRecords" : 0,
   "shuffleWriteBytes" : 13180,
+  "shuffleWriteTime" : 692000,
   "shuffleWriteRecords" : 0,
-  "memoryBytesSpilled" : 0,
-  "diskBytesSpilled" : 0,
   "name" : "map at <console>:14",
   "details" : 
"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:14)\n$line10.$read$$iwC$$iwC$$iwC.<init>(<console>:19)\n$line10.$read$$iwC$$iwC.<init>(<console>:21)\n$line10.$read$$iwC.<init>(<console>:23)\n$line10.$read.<init>(<console>:25)\n$line10.$read$.<init>(<console>:29)\n$line10.$read$.<clinit>(<console>)\n$line10.$eval$.<init>(<console>:7)\n$line10.$eval$.<clinit>(<console>)\n$line10.$eval.$print(<console>)\nsun.reflect.NativeMethod
 [...]
   "schedulingPool" : "default",
@@ -70,21 +96,34 @@
   "numFailedTasks" : 0,
   "numKilledTasks" : 0,
   "numCompletedIndices" : 8,
-  "executorRunTime" : 4338,
-  "executorCpuTime" : 0,
   "submissionTime" : "2015-02-03T16:43:04.228GMT",
   "firstTaskLaunchedTime" : "2015-02-03T16:43:04.234GMT",
   "completionTime" : "2015-02-03T16:43:04.819GMT",
+  "executorDeserializeTime" : 91,
+  "executorDeserializeCpuTime" : 0,
+  "executorRunTime" : 4338,
+  "executorCpuTime" : 0,
+  "resultSize" : 10144,
+  "jvmGcTime" : 200,
+  "resultSerializationTime" : 5,
+  "memoryBytesSpilled" : 0,
+  "diskBytesSpilled" : 0,
+  "peakExecutionMemory" : 0,
   "inputBytes" : 0,
   "inputRecords" : 0,
   "outputBytes" : 0,
   "outputRecords" : 0,
+  "shuffleRemoteBlocksFetched" : 0,
+  "shuffleLocalBlocksFetched" : 0,
+  "shuffleFetchWaitTime" : 0,
+  "shuffleRemoteBytesRead" : 0,
+  "shuffleRemoteBytesReadToDisk" : 0,
+  "shuffleLocalBytesRead" : 0,
   "shuffleReadBytes" : 0,
   "shuffleReadRecords" : 0,
   "shuffleWriteBytes" : 0,
+  "shuffleWriteTime" : 0,
   "shuffleWriteRecords" : 0,
-  "memoryBytesSpilled" : 0,
-  "diskBytesSpilled" : 0,
   "name" : "count at <console>:15",
   "details" : 
"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:15)\n$line9.$read$$iwC$$iwC$$iwC.<init>(<console>:20)\n$line9.$read$$iwC$$iwC.<init>(<console>:22)\n$line9.$read$$iwC.<init>(<console>:24)\n$line9.$read.<init>(<console>:26)\n$line9.$read$.<init>(<console>:30)\n$line9.$read$.<clinit>(<console>)\n$line9.$eval$.<init>(<console>:7)\n$line9.$eval$.<clinit>(<console>)\n$line9.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessor
 [...]
   "schedulingPool" : "default",
diff --git 
a/core/src/test/resources/HistoryServerExpectations/failed_stage_list_json_expectation.json
 
b/core/src/test/resources/HistoryServerExpectations/failed_stage_list_json_expectation.json
index 2fd5566..da26271 100644
--- 
a/core/src/test/resources/HistoryServerExpectations/failed_stage_list_json_expectation.json
+++ 
b/core/src/test/resources/HistoryServerExpectations/failed_stage_list_json_expectation.json
@@ -8,22 +8,35 @@
   "numFailedTasks" : 1,
   "numKilledTasks" : 0,
   "numCompletedIndices" : 7,
-  "executorRunTime" : 278,
-  "executorCpuTime" : 0,
   "submissionTime" : "2015-02-03T16:43:06.296GMT",
   "firstTaskLaunchedTime" : "2015-02-03T16:43:06.296GMT",
   "completionTime" : "2015-02-03T16:43:06.347GMT",
   "failureReason" : "Job aborted due to stage failure: Task 3 in stage 2.0 
failed 1 times, most recent failure: Lost task 3.0 in stage 2.0 (TID 19, 
localhost): java.lang.RuntimeException: got a 3, failing\n\tat 
$line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(<console>:18)\n\tat 
$line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(<console>:17)\n\tat 
scala.collection.Iterator$$anon$11.next(Iterator.scala:328)\n\tat 
org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1311)\n\tat org. [...]
+  "executorDeserializeTime" : 10,
+  "executorDeserializeCpuTime" : 0,
+  "executorRunTime" : 278,
+  "executorCpuTime" : 0,
+  "resultSize" : 6034,
+  "jvmGcTime" : 0,
+  "resultSerializationTime" : 0,
+  "memoryBytesSpilled" : 0,
+  "diskBytesSpilled" : 0,
+  "peakExecutionMemory" : 0,
   "inputBytes" : 0,
   "inputRecords" : 0,
   "outputBytes" : 0,
   "outputRecords" : 0,
+  "shuffleRemoteBlocksFetched" : 0,
+  "shuffleLocalBlocksFetched" : 64,
+  "shuffleFetchWaitTime" : 1,
+  "shuffleRemoteBytesRead" : 0,
+  "shuffleRemoteBytesReadToDisk" : 0,
+  "shuffleLocalBytesRead" : 0,
   "shuffleReadBytes" : 0,
   "shuffleReadRecords" : 0,
   "shuffleWriteBytes" : 0,
+  "shuffleWriteTime" : 0,
   "shuffleWriteRecords" : 0,
-  "memoryBytesSpilled" : 0,
-  "diskBytesSpilled" : 0,
   "name" : "count at <console>:20",
   "details" : 
"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:20)\n$line11.$read$$iwC$$iwC$$iwC.<init>(<console>:25)\n$line11.$read$$iwC$$iwC.<init>(<console>:27)\n$line11.$read$$iwC.<init>(<console>:29)\n$line11.$read.<init>(<console>:31)\n$line11.$read$.<init>(<console>:35)\n$line11.$read$.<clinit>(<console>)\n$line11.$eval$.<init>(<console>:7)\n$line11.$eval$.<clinit>(<console>)\n$line11.$eval.$print(<console>)\nsun.reflect.NativeMeth
 [...]
   "schedulingPool" : "default",
diff --git 
a/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json
 
b/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json
index dfa7385..7919070 100644
--- 
a/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json
+++ 
b/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json
@@ -8,21 +8,34 @@
   "numFailedTasks" : 0,
   "numKilledTasks" : 0,
   "numCompletedIndices" : 8,
-  "executorRunTime" : 3476,
-  "executorCpuTime" : 0,
   "submissionTime" : "2015-02-03T16:43:05.829GMT",
   "firstTaskLaunchedTime" : "2015-02-03T16:43:05.829GMT",
   "completionTime" : "2015-02-03T16:43:06.286GMT",
+  "executorDeserializeTime" : 13,
+  "executorDeserializeCpuTime" : 0,
+  "executorRunTime" : 3476,
+  "executorCpuTime" : 0,
+  "resultSize" : 15216,
+  "jvmGcTime" : 152,
+  "resultSerializationTime" : 9,
+  "memoryBytesSpilled" : 0,
+  "diskBytesSpilled" : 0,
+  "peakExecutionMemory" : 0,
   "inputBytes" : 28000128,
   "inputRecords" : 0,
   "outputBytes" : 0,
   "outputRecords" : 0,
+  "shuffleRemoteBlocksFetched" : 0,
+  "shuffleLocalBlocksFetched" : 0,
+  "shuffleFetchWaitTime" : 0,
+  "shuffleRemoteBytesRead" : 0,
+  "shuffleRemoteBytesReadToDisk" : 0,
+  "shuffleLocalBytesRead" : 0,
   "shuffleReadBytes" : 0,
   "shuffleReadRecords" : 0,
   "shuffleWriteBytes" : 13180,
+  "shuffleWriteTime" : 692000,
   "shuffleWriteRecords" : 0,
-  "memoryBytesSpilled" : 0,
-  "diskBytesSpilled" : 0,
   "name" : "map at <console>:14",
   "details" : 
"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:14)\n$line10.$read$$iwC$$iwC$$iwC.<init>(<console>:19)\n$line10.$read$$iwC$$iwC.<init>(<console>:21)\n$line10.$read$$iwC.<init>(<console>:23)\n$line10.$read.<init>(<console>:25)\n$line10.$read$.<init>(<console>:29)\n$line10.$read$.<clinit>(<console>)\n$line10.$eval$.<init>(<console>:7)\n$line10.$eval$.<clinit>(<console>)\n$line10.$eval.$print(<console>)\nsun.reflect.NativeMethod
 [...]
   "schedulingPool" : "default",
diff --git 
a/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json
 
b/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json
index e036627..50d3f74 100644
--- 
a/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json
+++ 
b/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json
@@ -8,21 +8,34 @@
   "numFailedTasks" : 0,
   "numKilledTasks" : 0,
   "numCompletedIndices" : 8,
-  "executorRunTime" : 3476,
-  "executorCpuTime" : 0,
   "submissionTime" : "2015-02-03T16:43:05.829GMT",
   "firstTaskLaunchedTime" : "2015-02-03T16:43:05.829GMT",
   "completionTime" : "2015-02-03T16:43:06.286GMT",
+  "executorDeserializeTime" : 13,
+  "executorDeserializeCpuTime" : 0,
+  "executorRunTime" : 3476,
+  "executorCpuTime" : 0,
+  "resultSize" : 15216,
+  "jvmGcTime" : 152,
+  "resultSerializationTime" : 9,
+  "memoryBytesSpilled" : 0,
+  "diskBytesSpilled" : 0,
+  "peakExecutionMemory" : 0,
   "inputBytes" : 28000128,
   "inputRecords" : 0,
   "outputBytes" : 0,
   "outputRecords" : 0,
+  "shuffleRemoteBlocksFetched" : 0,
+  "shuffleLocalBlocksFetched" : 0,
+  "shuffleFetchWaitTime" : 0,
+  "shuffleRemoteBytesRead" : 0,
+  "shuffleRemoteBytesReadToDisk" : 0,
+  "shuffleLocalBytesRead" : 0,
   "shuffleReadBytes" : 0,
   "shuffleReadRecords" : 0,
   "shuffleWriteBytes" : 13180,
+  "shuffleWriteTime" : 692000,
   "shuffleWriteRecords" : 0,
-  "memoryBytesSpilled" : 0,
-  "diskBytesSpilled" : 0,
   "name" : "map at <console>:14",
   "details" : 
"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:14)\n$line10.$read$$iwC$$iwC$$iwC.<init>(<console>:19)\n$line10.$read$$iwC$$iwC.<init>(<console>:21)\n$line10.$read$$iwC.<init>(<console>:23)\n$line10.$read.<init>(<console>:25)\n$line10.$read$.<init>(<console>:29)\n$line10.$read$.<clinit>(<console>)\n$line10.$eval$.<init>(<console>:7)\n$line10.$eval$.<clinit>(<console>)\n$line10.$eval.$print(<console>)\nsun.reflect.NativeMethod
 [...]
   "schedulingPool" : "default",
diff --git 
a/core/src/test/resources/HistoryServerExpectations/stage_list_json_expectation.json
 
b/core/src/test/resources/HistoryServerExpectations/stage_list_json_expectation.json
index 1e6fb40..edbac71 100644
--- 
a/core/src/test/resources/HistoryServerExpectations/stage_list_json_expectation.json
+++ 
b/core/src/test/resources/HistoryServerExpectations/stage_list_json_expectation.json
@@ -8,21 +8,34 @@
   "numFailedTasks" : 0,
   "numKilledTasks" : 0,
   "numCompletedIndices" : 8,
-  "executorRunTime" : 162,
-  "executorCpuTime" : 0,
   "submissionTime" : "2015-02-03T16:43:07.191GMT",
   "firstTaskLaunchedTime" : "2015-02-03T16:43:07.191GMT",
   "completionTime" : "2015-02-03T16:43:07.226GMT",
+  "executorDeserializeTime" : 36,
+  "executorDeserializeCpuTime" : 0,
+  "executorRunTime" : 162,
+  "executorCpuTime" : 0,
+  "resultSize" : 14496,
+  "jvmGcTime" : 0,
+  "resultSerializationTime" : 1,
+  "memoryBytesSpilled" : 0,
+  "diskBytesSpilled" : 0,
+  "peakExecutionMemory" : 0,
   "inputBytes" : 160,
   "inputRecords" : 0,
   "outputBytes" : 0,
   "outputRecords" : 0,
+  "shuffleRemoteBlocksFetched" : 0,
+  "shuffleLocalBlocksFetched" : 0,
+  "shuffleFetchWaitTime" : 0,
+  "shuffleRemoteBytesRead" : 0,
+  "shuffleRemoteBytesReadToDisk" : 0,
+  "shuffleLocalBytesRead" : 0,
   "shuffleReadBytes" : 0,
   "shuffleReadRecords" : 0,
   "shuffleWriteBytes" : 0,
+  "shuffleWriteTime" : 0,
   "shuffleWriteRecords" : 0,
-  "memoryBytesSpilled" : 0,
-  "diskBytesSpilled" : 0,
   "name" : "count at <console>:17",
   "details" : 
"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line19.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:17)\n$line19.$read$$iwC$$iwC$$iwC.<init>(<console>:22)\n$line19.$read$$iwC$$iwC.<init>(<console>:24)\n$line19.$read$$iwC.<init>(<console>:26)\n$line19.$read.<init>(<console>:28)\n$line19.$read$.<init>(<console>:32)\n$line19.$read$.<clinit>(<console>)\n$line19.$eval$.<init>(<console>:7)\n$line19.$eval$.<clinit>(<console>)\n$line19.$eval.$print(<console>)\nsun.reflect.NativeMeth
 [...]
   "schedulingPool" : "default",
@@ -39,22 +52,35 @@
   "numFailedTasks" : 1,
   "numKilledTasks" : 0,
   "numCompletedIndices" : 7,
-  "executorRunTime" : 278,
-  "executorCpuTime" : 0,
   "submissionTime" : "2015-02-03T16:43:06.296GMT",
   "firstTaskLaunchedTime" : "2015-02-03T16:43:06.296GMT",
   "completionTime" : "2015-02-03T16:43:06.347GMT",
   "failureReason" : "Job aborted due to stage failure: Task 3 in stage 2.0 
failed 1 times, most recent failure: Lost task 3.0 in stage 2.0 (TID 19, 
localhost): java.lang.RuntimeException: got a 3, failing\n\tat 
$line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(<console>:18)\n\tat 
$line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(<console>:17)\n\tat 
scala.collection.Iterator$$anon$11.next(Iterator.scala:328)\n\tat 
org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1311)\n\tat org. [...]
+  "executorDeserializeTime" : 10,
+  "executorDeserializeCpuTime" : 0,
+  "executorRunTime" : 278,
+  "executorCpuTime" : 0,
+  "resultSize" : 6034,
+  "jvmGcTime" : 0,
+  "resultSerializationTime" : 0,
+  "memoryBytesSpilled" : 0,
+  "diskBytesSpilled" : 0,
+  "peakExecutionMemory" : 0,
   "inputBytes" : 0,
   "inputRecords" : 0,
   "outputBytes" : 0,
   "outputRecords" : 0,
+  "shuffleRemoteBlocksFetched" : 0,
+  "shuffleLocalBlocksFetched" : 64,
+  "shuffleFetchWaitTime" : 1,
+  "shuffleRemoteBytesRead" : 0,
+  "shuffleRemoteBytesReadToDisk" : 0,
+  "shuffleLocalBytesRead" : 0,
   "shuffleReadBytes" : 0,
   "shuffleReadRecords" : 0,
   "shuffleWriteBytes" : 0,
+  "shuffleWriteTime" : 0,
   "shuffleWriteRecords" : 0,
-  "memoryBytesSpilled" : 0,
-  "diskBytesSpilled" : 0,
   "name" : "count at <console>:20",
   "details" : 
"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:20)\n$line11.$read$$iwC$$iwC$$iwC.<init>(<console>:25)\n$line11.$read$$iwC$$iwC.<init>(<console>:27)\n$line11.$read$$iwC.<init>(<console>:29)\n$line11.$read.<init>(<console>:31)\n$line11.$read$.<init>(<console>:35)\n$line11.$read$.<clinit>(<console>)\n$line11.$eval$.<init>(<console>:7)\n$line11.$eval$.<clinit>(<console>)\n$line11.$eval.$print(<console>)\nsun.reflect.NativeMeth
 [...]
   "schedulingPool" : "default",
@@ -71,21 +97,34 @@
   "numFailedTasks" : 0,
   "numKilledTasks" : 0,
   "numCompletedIndices" : 8,
-  "executorRunTime" : 3476,
-  "executorCpuTime" : 0,
   "submissionTime" : "2015-02-03T16:43:05.829GMT",
   "firstTaskLaunchedTime" : "2015-02-03T16:43:05.829GMT",
   "completionTime" : "2015-02-03T16:43:06.286GMT",
+  "executorDeserializeTime" : 13,
+  "executorDeserializeCpuTime" : 0,
+  "executorRunTime" : 3476,
+  "executorCpuTime" : 0,
+  "resultSize" : 15216,
+  "jvmGcTime" : 152,
+  "resultSerializationTime" : 9,
+  "memoryBytesSpilled" : 0,
+  "diskBytesSpilled" : 0,
+  "peakExecutionMemory" : 0,
   "inputBytes" : 28000128,
   "inputRecords" : 0,
   "outputBytes" : 0,
   "outputRecords" : 0,
+  "shuffleRemoteBlocksFetched" : 0,
+  "shuffleLocalBlocksFetched" : 0,
+  "shuffleFetchWaitTime" : 0,
+  "shuffleRemoteBytesRead" : 0,
+  "shuffleRemoteBytesReadToDisk" : 0,
+  "shuffleLocalBytesRead" : 0,
   "shuffleReadBytes" : 0,
   "shuffleReadRecords" : 0,
   "shuffleWriteBytes" : 13180,
+  "shuffleWriteTime" : 692000,
   "shuffleWriteRecords" : 0,
-  "memoryBytesSpilled" : 0,
-  "diskBytesSpilled" : 0,
   "name" : "map at <console>:14",
   "details" : 
"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:14)\n$line10.$read$$iwC$$iwC$$iwC.<init>(<console>:19)\n$line10.$read$$iwC$$iwC.<init>(<console>:21)\n$line10.$read$$iwC.<init>(<console>:23)\n$line10.$read.<init>(<console>:25)\n$line10.$read$.<init>(<console>:29)\n$line10.$read$.<clinit>(<console>)\n$line10.$eval$.<init>(<console>:7)\n$line10.$eval$.<clinit>(<console>)\n$line10.$eval.$print(<console>)\nsun.reflect.NativeMethod
 [...]
   "schedulingPool" : "default",
@@ -102,21 +141,34 @@
   "numFailedTasks" : 0,
   "numKilledTasks" : 0,
   "numCompletedIndices" : 8,
-  "executorRunTime" : 4338,
-  "executorCpuTime" : 0,
   "submissionTime" : "2015-02-03T16:43:04.228GMT",
   "firstTaskLaunchedTime" : "2015-02-03T16:43:04.234GMT",
   "completionTime" : "2015-02-03T16:43:04.819GMT",
+  "executorDeserializeTime" : 91,
+  "executorDeserializeCpuTime" : 0,
+  "executorRunTime" : 4338,
+  "executorCpuTime" : 0,
+  "resultSize" : 10144,
+  "jvmGcTime" : 200,
+  "resultSerializationTime" : 5,
+  "memoryBytesSpilled" : 0,
+  "diskBytesSpilled" : 0,
+  "peakExecutionMemory" : 0,
   "inputBytes" : 0,
   "inputRecords" : 0,
   "outputBytes" : 0,
   "outputRecords" : 0,
+  "shuffleRemoteBlocksFetched" : 0,
+  "shuffleLocalBlocksFetched" : 0,
+  "shuffleFetchWaitTime" : 0,
+  "shuffleRemoteBytesRead" : 0,
+  "shuffleRemoteBytesReadToDisk" : 0,
+  "shuffleLocalBytesRead" : 0,
   "shuffleReadBytes" : 0,
   "shuffleReadRecords" : 0,
   "shuffleWriteBytes" : 0,
+  "shuffleWriteTime" : 0,
   "shuffleWriteRecords" : 0,
-  "memoryBytesSpilled" : 0,
-  "diskBytesSpilled" : 0,
   "name" : "count at <console>:15",
   "details" : 
"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:15)\n$line9.$read$$iwC$$iwC$$iwC.<init>(<console>:20)\n$line9.$read$$iwC$$iwC.<init>(<console>:22)\n$line9.$read$$iwC.<init>(<console>:24)\n$line9.$read.<init>(<console>:26)\n$line9.$read$.<init>(<console>:30)\n$line9.$read$.<clinit>(<console>)\n$line9.$eval$.<init>(<console>:7)\n$line9.$eval$.<clinit>(<console>)\n$line9.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessor
 [...]
   "schedulingPool" : "default",
diff --git 
a/core/src/test/resources/HistoryServerExpectations/stage_list_with_accumulable_json_expectation.json
 
b/core/src/test/resources/HistoryServerExpectations/stage_list_with_accumulable_json_expectation.json
index e6284cc..836f2cb 100644
--- 
a/core/src/test/resources/HistoryServerExpectations/stage_list_with_accumulable_json_expectation.json
+++ 
b/core/src/test/resources/HistoryServerExpectations/stage_list_with_accumulable_json_expectation.json
@@ -8,21 +8,34 @@
   "numFailedTasks" : 0,
   "numKilledTasks" : 0,
   "numCompletedIndices" : 8,
-  "executorRunTime" : 120,
-  "executorCpuTime" : 0,
   "submissionTime" : "2015-03-16T19:25:36.103GMT",
   "firstTaskLaunchedTime" : "2015-03-16T19:25:36.515GMT",
   "completionTime" : "2015-03-16T19:25:36.579GMT",
+  "executorDeserializeTime" : 102,
+  "executorDeserializeCpuTime" : 0,
+  "executorRunTime" : 120,
+  "executorCpuTime" : 0,
+  "resultSize" : 5576,
+  "jvmGcTime" : 0,
+  "resultSerializationTime" : 15,
+  "memoryBytesSpilled" : 0,
+  "diskBytesSpilled" : 0,
+  "peakExecutionMemory" : 0,
   "inputBytes" : 0,
   "inputRecords" : 0,
   "outputBytes" : 0,
   "outputRecords" : 0,
+  "shuffleRemoteBlocksFetched" : 0,
+  "shuffleLocalBlocksFetched" : 0,
+  "shuffleFetchWaitTime" : 0,
+  "shuffleRemoteBytesRead" : 0,
+  "shuffleRemoteBytesReadToDisk" : 0,
+  "shuffleLocalBytesRead" : 0,
   "shuffleReadBytes" : 0,
   "shuffleReadRecords" : 0,
   "shuffleWriteBytes" : 0,
+  "shuffleWriteTime" : 0,
   "shuffleWriteRecords" : 0,
-  "memoryBytesSpilled" : 0,
-  "diskBytesSpilled" : 0,
   "name" : "foreach at <console>:15",
   "details" : 
"org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:15)\n$line9.$read$$iwC$$iwC$$iwC.<init>(<console>:20)\n$line9.$read$$iwC$$iwC.<init>(<console>:22)\n$line9.$read$$iwC.<init>(<console>:24)\n$line9.$read.<init>(<console>:26)\n$line9.$read$.<init>(<console>:30)\n$line9.$read$.<clinit>(<console>)\n$line9.$eval$.<init>(<console>:7)\n$line9.$eval$.<clinit>(<console>)\n$line9.$eval.$print(<console>)\nsun.reflect.NativeMethodAccess
 [...]
   "schedulingPool" : "default",
diff --git 
a/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json
 
b/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json
index b79f8a5..735a825 100644
--- 
a/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json
+++ 
b/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json
@@ -8,21 +8,34 @@
   "numFailedTasks" : 0,
   "numKilledTasks" : 0,
   "numCompletedIndices" : 8,
-  "executorRunTime" : 120,
-  "executorCpuTime" : 0,
   "submissionTime" : "2015-03-16T19:25:36.103GMT",
   "firstTaskLaunchedTime" : "2015-03-16T19:25:36.515GMT",
   "completionTime" : "2015-03-16T19:25:36.579GMT",
+  "executorDeserializeTime" : 102,
+  "executorDeserializeCpuTime" : 0,
+  "executorRunTime" : 120,
+  "executorCpuTime" : 0,
+  "resultSize" : 5576,
+  "jvmGcTime" : 0,
+  "resultSerializationTime" : 15,
+  "memoryBytesSpilled" : 0,
+  "diskBytesSpilled" : 0,
+  "peakExecutionMemory" : 0,
   "inputBytes" : 0,
   "inputRecords" : 0,
   "outputBytes" : 0,
   "outputRecords" : 0,
+  "shuffleRemoteBlocksFetched" : 0,
+  "shuffleLocalBlocksFetched" : 0,
+  "shuffleFetchWaitTime" : 0,
+  "shuffleRemoteBytesRead" : 0,
+  "shuffleRemoteBytesReadToDisk" : 0,
+  "shuffleLocalBytesRead" : 0,
   "shuffleReadBytes" : 0,
   "shuffleReadRecords" : 0,
   "shuffleWriteBytes" : 0,
+  "shuffleWriteTime" : 0,
   "shuffleWriteRecords" : 0,
-  "memoryBytesSpilled" : 0,
-  "diskBytesSpilled" : 0,
   "name" : "foreach at <console>:15",
   "details" : 
"org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.<init>(<console>:15)\n$line9.$read$$iwC$$iwC$$iwC.<init>(<console>:20)\n$line9.$read$$iwC$$iwC.<init>(<console>:22)\n$line9.$read$$iwC.<init>(<console>:24)\n$line9.$read.<init>(<console>:26)\n$line9.$read$.<init>(<console>:30)\n$line9.$read$.<clinit>(<console>)\n$line9.$eval$.<init>(<console>:7)\n$line9.$eval$.<clinit>(<console>)\n$line9.$eval.$print(<console>)\nsun.reflect.NativeMethodAccess
 [...]
   "schedulingPool" : "default",
diff --git a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala 
b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala
index 5e976ae..f489698 100644
--- a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala
@@ -51,23 +51,36 @@ class StagePageSuite extends SparkFunSuite with 
LocalSparkContext {
         numKilledTasks = 1,
         numCompletedIndices = 1,
 
-        executorRunTime = 1L,
-        executorCpuTime = 1L,
         submissionTime = None,
         firstTaskLaunchedTime = None,
         completionTime = None,
         failureReason = None,
 
+        executorDeserializeTime = 1L,
+        executorDeserializeCpuTime = 1L,
+        executorRunTime = 1L,
+        executorCpuTime = 1L,
+        resultSize = 1L,
+        jvmGcTime = 1L,
+        resultSerializationTime = 1L,
+        memoryBytesSpilled = 1L,
+        diskBytesSpilled = 1L,
+        peakExecutionMemory = 1L,
         inputBytes = 1L,
         inputRecords = 1L,
         outputBytes = 1L,
         outputRecords = 1L,
+        shuffleRemoteBlocksFetched = 1L,
+        shuffleLocalBlocksFetched = 1L,
+        shuffleFetchWaitTime = 1L,
+        shuffleRemoteBytesRead = 1L,
+        shuffleRemoteBytesReadToDisk = 1L,
+        shuffleLocalBytesRead = 1L,
         shuffleReadBytes = 1L,
         shuffleReadRecords = 1L,
         shuffleWriteBytes = 1L,
+        shuffleWriteTime = 1L,
         shuffleWriteRecords = 1L,
-        memoryBytesSpilled = 1L,
-        diskBytesSpilled = 1L,
 
         name = "stage1",
         description = Some("description"),


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to