You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by ir...@apache.org on 2019/03/04 20:26:11 UTC

[spark] branch master updated: [SPARK-25865][CORE] Add GC information to ExecutorMetrics

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

irashid 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 e5c502c  [SPARK-25865][CORE] Add GC information to ExecutorMetrics
e5c502c is described below

commit e5c502c596563dce8eb58f86e42c1aea2c51ed17
Author: LantaoJin <ji...@gmail.com>
AuthorDate: Mon Mar 4 14:26:02 2019 -0600

    [SPARK-25865][CORE] Add GC information to ExecutorMetrics
    
    ## What changes were proposed in this pull request?
    
    Only memory usage without GC information could not help us to determinate the proper settings of memory. We need the GC metrics about frequency of major & minor GC. For example, two cases, their configured memory for executor are all 10GB and their usages are all near 10GB. So should we increase or decrease the configured memory for them? This metrics may be helpful. We can increase configured memory for the first one if it has very frequency major GC and decrease the second one if on [...]
    GC metrics are only useful in entire lifetime of executors instead of separated stages.
    
    ## How was this patch tested?
    
    Adding UT.
    
    Closes #22874 from LantaoJin/SPARK-25865.
    
    Authored-by: LantaoJin <ji...@gmail.com>
    Signed-off-by: Imran Rashid <ir...@cloudera.com>
---
 .../org/apache/spark/internal/config/package.scala |  19 ++
 .../apache/spark/metrics/ExecutorMetricType.scala  |  63 +++++-
 .../application_list_json_expectation.json         |  18 ++
 .../completed_app_list_json_expectation.json       |  18 ++
 ...arbage_collection_metrics_json_expectation.json | 122 ++++++++++++
 ...ist_with_executor_metrics_json_expectation.json |  30 ++-
 ...utor_process_tree_metrics_json_expectation.json |  12 +-
 .../limit_app_list_json_expectation.json           |  33 ++--
 .../minDate_app_list_json_expectation.json         |  18 ++
 .../minEndDate_app_list_json_expectation.json      |  18 ++
 .../spark-events/application_1536831636016_59384_1 | 214 +++++++++++++++++++++
 .../spark/deploy/history/HistoryServerSuite.scala  |   2 +
 .../scheduler/EventLoggingListenerSuite.scala      |  36 ++--
 .../org/apache/spark/util/JsonProtocolSuite.scala  |  18 +-
 dev/.rat-excludes                                  |   1 +
 15 files changed, 576 insertions(+), 46 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala
index 3ef6cba..d6a359d 100644
--- a/core/src/main/scala/org/apache/spark/internal/config/package.scala
+++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala
@@ -20,6 +20,7 @@ package org.apache.spark.internal
 import java.util.concurrent.TimeUnit
 
 import org.apache.spark.launcher.SparkLauncher
+import org.apache.spark.metrics.GarbageCollectionMetrics
 import org.apache.spark.network.util.ByteUnit
 import org.apache.spark.scheduler.{EventLoggingListener, SchedulingMode}
 import org.apache.spark.storage.{DefaultTopologyMapper, RandomBlockReplicationPolicy}
@@ -114,6 +115,24 @@ package object config {
       .booleanConf
       .createWithDefault(false)
 
+  private[spark] val EVENT_LOG_GC_METRICS_YOUNG_GENERATION_GARBAGE_COLLECTORS =
+    ConfigBuilder("spark.eventLog.gcMetrics.youngGenerationGarbageCollectors")
+      .doc("Names of supported young generation garbage collector. A name usually is " +
+        " the return of GarbageCollectorMXBean.getName. The built-in young generation garbage " +
+        s"collectors are ${GarbageCollectionMetrics.YOUNG_GENERATION_BUILTIN_GARBAGE_COLLECTORS}")
+      .stringConf
+      .toSequence
+      .createWithDefault(GarbageCollectionMetrics.YOUNG_GENERATION_BUILTIN_GARBAGE_COLLECTORS)
+
+  private[spark] val EVENT_LOG_GC_METRICS_OLD_GENERATION_GARBAGE_COLLECTORS =
+    ConfigBuilder("spark.eventLog.gcMetrics.oldGenerationGarbageCollectors")
+      .doc("Names of supported old generation garbage collector. A name usually is " +
+        "the return of GarbageCollectorMXBean.getName. The built-in old generation garbage " +
+        s"collectors are ${GarbageCollectionMetrics.OLD_GENERATION_BUILTIN_GARBAGE_COLLECTORS}")
+      .stringConf
+      .toSequence
+      .createWithDefault(GarbageCollectionMetrics.OLD_GENERATION_BUILTIN_GARBAGE_COLLECTORS)
+
   private[spark] val EVENT_LOG_OVERWRITE =
     ConfigBuilder("spark.eventLog.overwrite").booleanConf.createWithDefault(false)
 
diff --git a/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala b/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala
index 704b36d..6d8e9a8 100644
--- a/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala
+++ b/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala
@@ -19,9 +19,12 @@ package org.apache.spark.metrics
 import java.lang.management.{BufferPoolMXBean, ManagementFactory}
 import javax.management.ObjectName
 
+import scala.collection.JavaConverters._
 import scala.collection.mutable
 
+import org.apache.spark.SparkEnv
 import org.apache.spark.executor.ProcfsMetricsGetter
+import org.apache.spark.internal.{config, Logging}
 import org.apache.spark.memory.MemoryManager
 
 /**
@@ -99,6 +102,63 @@ case object ProcessTreeMetrics extends ExecutorMetricType {
   }
 }
 
+case object GarbageCollectionMetrics extends ExecutorMetricType with Logging {
+  private var nonBuiltInCollectors: Seq[String] = Nil
+
+  override val names = Seq(
+    "MinorGCCount",
+    "MinorGCTime",
+    "MajorGCCount",
+    "MajorGCTime"
+  )
+
+  /* We builtin some common GC collectors which categorized as young generation and old */
+  private[spark] val YOUNG_GENERATION_BUILTIN_GARBAGE_COLLECTORS = Seq(
+    "Copy",
+    "PS Scavenge",
+    "ParNew",
+    "G1 Young Generation"
+  )
+
+  private[spark] val OLD_GENERATION_BUILTIN_GARBAGE_COLLECTORS = Seq(
+    "MarkSweepCompact",
+    "PS MarkSweep",
+    "ConcurrentMarkSweep",
+    "G1 Old Generation"
+  )
+
+  private lazy val youngGenerationGarbageCollector: Seq[String] = {
+    SparkEnv.get.conf.get(config.EVENT_LOG_GC_METRICS_YOUNG_GENERATION_GARBAGE_COLLECTORS)
+  }
+
+  private lazy val oldGenerationGarbageCollector: Seq[String] = {
+    SparkEnv.get.conf.get(config.EVENT_LOG_GC_METRICS_OLD_GENERATION_GARBAGE_COLLECTORS)
+  }
+
+  override private[spark] def getMetricValues(memoryManager: MemoryManager): Array[Long] = {
+    val gcMetrics = new Array[Long](names.length) // minorCount, minorTime, majorCount, majorTime
+    ManagementFactory.getGarbageCollectorMXBeans.asScala.foreach { mxBean =>
+      if (youngGenerationGarbageCollector.contains(mxBean.getName)) {
+        gcMetrics(0) = mxBean.getCollectionCount
+        gcMetrics(1) = mxBean.getCollectionTime
+      } else if (oldGenerationGarbageCollector.contains(mxBean.getName)) {
+        gcMetrics(2) = mxBean.getCollectionCount
+        gcMetrics(3) = mxBean.getCollectionTime
+      } else if (!nonBuiltInCollectors.contains(mxBean.getName)) {
+        nonBuiltInCollectors = mxBean.getName +: nonBuiltInCollectors
+        // log it when first seen
+        logWarning(s"To enable non-built-in garbage collector(s) " +
+          s"$nonBuiltInCollectors, users should configure it(them) to " +
+          s"${config.EVENT_LOG_GC_METRICS_YOUNG_GENERATION_GARBAGE_COLLECTORS.key} or " +
+          s"${config.EVENT_LOG_GC_METRICS_OLD_GENERATION_GARBAGE_COLLECTORS.key}")
+      } else {
+        // do nothing
+      }
+    }
+    gcMetrics
+  }
+}
+
 case object OnHeapExecutionMemory extends MemoryManagerExecutorMetricType(
   _.onHeapExecutionMemoryUsed)
 
@@ -137,7 +197,8 @@ private[spark] object ExecutorMetricType {
     OffHeapUnifiedMemory,
     DirectPoolMemory,
     MappedPoolMemory,
-    ProcessTreeMetrics
+    ProcessTreeMetrics,
+    GarbageCollectionMetrics
   )
 
 
diff --git a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json
index 0f0ccf9..71ccedc 100644
--- a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json
@@ -1,4 +1,22 @@
 [ {
+  "id": "application_1536831636016_59384",
+  "name": "Spark Pi",
+  "attempts": [
+    {
+      "attemptId": "1",
+      "startTime": "2019-01-08T04:33:43.607GMT",
+      "endTime": "2019-01-08T04:33:58.745GMT",
+      "lastUpdated": "",
+      "duration": 15138,
+      "sparkUser": "lajin",
+      "completed": true,
+      "appSparkVersion": "3.0.0-SNAPSHOT",
+      "lastUpdatedEpoch": 0,
+      "startTimeEpoch": 1546922023607,
+      "endTimeEpoch": 1546922038745
+    }
+  ]
+}, {
   "id" : "application_1538416563558_0014",
   "name" : "PythonBisectingKMeansExample",
   "attempts" : [ {
diff --git a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json
index e136a35..ad5f0ea 100644
--- a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json
@@ -1,4 +1,22 @@
 [ {
+  "id": "application_1536831636016_59384",
+  "name": "Spark Pi",
+  "attempts": [
+    {
+      "attemptId": "1",
+      "startTime": "2019-01-08T04:33:43.607GMT",
+      "endTime": "2019-01-08T04:33:58.745GMT",
+      "lastUpdated": "",
+      "duration": 15138,
+      "sparkUser": "lajin",
+      "completed": true,
+      "appSparkVersion": "3.0.0-SNAPSHOT",
+      "lastUpdatedEpoch": 0,
+      "startTimeEpoch": 1546922023607,
+      "endTimeEpoch": 1546922038745
+    }
+  ]
+}, {
   "id" : "application_1538416563558_0014",
   "name" : "PythonBisectingKMeansExample",
   "attempts" : [ {
diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_garbage_collection_metrics_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_garbage_collection_metrics_json_expectation.json
new file mode 100644
index 0000000..f0f39e6
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_garbage_collection_metrics_json_expectation.json
@@ -0,0 +1,122 @@
+[ {
+  "id" : "driver",
+  "hostPort" : "047.company.com:42509",
+  "isActive" : true,
+  "rddBlocks" : 0,
+  "memoryUsed" : 0,
+  "diskUsed" : 0,
+  "totalCores" : 0,
+  "maxTasks" : 0,
+  "activeTasks" : 0,
+  "failedTasks" : 0,
+  "completedTasks" : 0,
+  "totalTasks" : 0,
+  "totalDuration" : 0,
+  "totalGCTime" : 0,
+  "totalInputBytes" : 0,
+  "totalShuffleRead" : 0,
+  "totalShuffleWrite" : 0,
+  "isBlacklisted" : false,
+  "maxMemory" : 100977868,
+  "addTime" : "2019-01-08T04:33:44.502GMT",
+  "executorLogs" : {
+    "stdout" : "https://047.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000001/lajin/stdout?start=-4096",
+    "stderr" : "https://047.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000001/lajin/stderr?start=-4096"
+  },
+  "memoryMetrics" : {
+    "usedOnHeapStorageMemory" : 0,
+    "usedOffHeapStorageMemory" : 0,
+    "totalOnHeapStorageMemory" : 100977868,
+    "totalOffHeapStorageMemory" : 0
+  },
+  "blacklistedInStages" : [ ],
+  "peakMemoryMetrics" : {
+    "JVMHeapMemory" : 211171816,
+    "JVMOffHeapMemory" : 90237256,
+    "OnHeapExecutionMemory" : 0,
+    "OffHeapExecutionMemory" : 0,
+    "OnHeapStorageMemory" : 4876,
+    "OffHeapStorageMemory" : 0,
+    "OnHeapUnifiedMemory" : 4876,
+    "OffHeapUnifiedMemory" : 0,
+    "DirectPoolMemory" : 806275,
+    "MappedPoolMemory" : 0,
+    "ProcessTreeJVMVMemory" : 2646888448,
+    "ProcessTreeJVMRSSMemory" : 520900608,
+    "ProcessTreePythonVMemory" : 0,
+    "ProcessTreePythonRSSMemory" : 0,
+    "ProcessTreeOtherVMemory" : 0,
+    "ProcessTreeOtherRSSMemory" : 0,
+    "MinorGCCount" : 8,
+    "MinorGCTime" : 374,
+    "MajorGCCount" : 3,
+    "MajorGCTime" : 170
+  },
+  "attributes" : { }
+}, {
+  "id" : "2",
+  "hostPort" : "028.company.com:46325",
+  "isActive" : true,
+  "rddBlocks" : 0,
+  "memoryUsed" : 0,
+  "diskUsed" : 0,
+  "totalCores" : 4,
+  "maxTasks" : 4,
+  "activeTasks" : 0,
+  "failedTasks" : 0,
+  "completedTasks" : 52,
+  "totalTasks" : 52,
+  "totalDuration" : 8879,
+  "totalGCTime" : 420,
+  "totalInputBytes" : 0,
+  "totalShuffleRead" : 0,
+  "totalShuffleWrite" : 0,
+  "isBlacklisted" : false,
+  "maxMemory" : 97832140,
+  "addTime" : "2019-01-08T04:33:54.270GMT",
+  "executorLogs" : {
+    "stdout" : "https://028.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000003/lajin/stdout?start=-4096",
+    "stderr" : "https://028.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000003/lajin/stderr?start=-4096"
+  },
+  "memoryMetrics" : {
+    "usedOnHeapStorageMemory" : 0,
+    "usedOffHeapStorageMemory" : 0,
+    "totalOnHeapStorageMemory" : 97832140,
+    "totalOffHeapStorageMemory" : 0
+  },
+  "blacklistedInStages" : [ ],
+  "attributes" : { }
+}, {
+  "id" : "1",
+  "hostPort" : "036.company.com:35126",
+  "isActive" : true,
+  "rddBlocks" : 0,
+  "memoryUsed" : 0,
+  "diskUsed" : 0,
+  "totalCores" : 4,
+  "maxTasks" : 4,
+  "activeTasks" : 0,
+  "failedTasks" : 0,
+  "completedTasks" : 48,
+  "totalTasks" : 48,
+  "totalDuration" : 8837,
+  "totalGCTime" : 1192,
+  "totalInputBytes" : 0,
+  "totalShuffleRead" : 0,
+  "totalShuffleWrite" : 0,
+  "isBlacklisted" : false,
+  "maxMemory" : 97832140,
+  "addTime" : "2019-01-08T04:33:55.929GMT",
+  "executorLogs" : {
+    "stdout" : "https://036.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000002/lajin/stdout?start=-4096",
+    "stderr" : "https://036.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000002/lajin/stderr?start=-4096"
+  },
+  "memoryMetrics" : {
+    "usedOnHeapStorageMemory" : 0,
+    "usedOffHeapStorageMemory" : 0,
+    "totalOnHeapStorageMemory" : 97832140,
+    "totalOffHeapStorageMemory" : 0
+  },
+  "blacklistedInStages" : [ ],
+  "attributes" : { }
+} ]
\ No newline at end of file
diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json
index f282d483..3db5377 100644
--- a/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json
@@ -43,7 +43,11 @@
     "ProcessTreePythonVMemory": 0,
     "ProcessTreePythonRSSMemory": 0,
     "ProcessTreeOtherVMemory": 0,
-    "ProcessTreeOtherRSSMemory": 0
+    "ProcessTreeOtherRSSMemory": 0,
+    "MinorGCCount": 0,
+    "MinorGCTime": 0,
+    "MajorGCCount": 0,
+    "MajorGCTime": 0
   },
   "attributes" : { }
 }, {
@@ -193,7 +197,11 @@
     "ProcessTreePythonVMemory": 0,
     "ProcessTreePythonRSSMemory": 0,
     "ProcessTreeOtherVMemory": 0,
-    "ProcessTreeOtherRSSMemory": 0
+    "ProcessTreeOtherRSSMemory": 0,
+    "MinorGCCount": 0,
+    "MinorGCTime": 0,
+    "MajorGCCount": 0,
+    "MajorGCTime": 0
   },
   "attributes" : { }
 }, {
@@ -244,7 +252,11 @@
     "ProcessTreePythonVMemory": 0,
     "ProcessTreePythonRSSMemory": 0,
     "ProcessTreeOtherVMemory": 0,
-    "ProcessTreeOtherRSSMemory": 0
+    "ProcessTreeOtherRSSMemory": 0,
+    "MinorGCCount": 0,
+    "MinorGCTime": 0,
+    "MajorGCCount": 0,
+    "MajorGCTime": 0
   },
   "attributes" : { }
 }, {
@@ -295,7 +307,11 @@
     "ProcessTreePythonVMemory": 0,
     "ProcessTreePythonRSSMemory": 0,
     "ProcessTreeOtherVMemory": 0,
-    "ProcessTreeOtherRSSMemory": 0
+    "ProcessTreeOtherRSSMemory": 0,
+    "MinorGCCount": 0,
+    "MinorGCTime": 0,
+    "MajorGCCount": 0,
+    "MajorGCTime": 0
   },
   "attributes" : { }
 }, {
@@ -346,7 +362,11 @@
     "ProcessTreePythonVMemory": 0,
     "ProcessTreePythonRSSMemory": 0,
     "ProcessTreeOtherVMemory": 0,
-    "ProcessTreeOtherRSSMemory": 0
+    "ProcessTreeOtherRSSMemory": 0,
+    "MinorGCCount": 0,
+    "MinorGCTime": 0,
+    "MajorGCCount": 0,
+    "MajorGCTime": 0
   },
   "attributes" : { }
 } ]
diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_process_tree_metrics_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_process_tree_metrics_json_expectation.json
index 980ec85..2c2efb5 100644
--- a/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_process_tree_metrics_json_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_process_tree_metrics_json_expectation.json
@@ -43,7 +43,11 @@
     "ProcessTreePythonVMemory" : 408375296,
     "ProcessTreePythonRSSMemory" : 40284160,
     "ProcessTreeOtherVMemory" : 0,
-    "ProcessTreeOtherRSSMemory" : 0
+    "ProcessTreeOtherRSSMemory" : 0,
+    "MinorGCCount": 0,
+    "MinorGCTime": 0,
+    "MajorGCCount": 0,
+    "MajorGCTime": 0
   },
   "attributes" : { }
 }, {
@@ -94,7 +98,11 @@
     "ProcessTreePythonVMemory" : 625926144,
     "ProcessTreePythonRSSMemory" : 69013504,
     "ProcessTreeOtherVMemory" : 0,
-    "ProcessTreeOtherRSSMemory" : 0
+    "ProcessTreeOtherRSSMemory" : 0,
+    "MinorGCCount": 0,
+    "MinorGCTime": 0,
+    "MajorGCCount": 0,
+    "MajorGCTime": 0
   },
   "attributes" : { }
 } ]
diff --git a/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json
index 0ef9377..c303b6c 100644
--- a/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json
@@ -1,4 +1,22 @@
 [ {
+  "id": "application_1536831636016_59384",
+  "name": "Spark Pi",
+  "attempts": [
+    {
+      "attemptId": "1",
+      "startTime": "2019-01-08T04:33:43.607GMT",
+      "endTime": "2019-01-08T04:33:58.745GMT",
+      "lastUpdated": "",
+      "duration": 15138,
+      "sparkUser": "lajin",
+      "completed": true,
+      "appSparkVersion": "3.0.0-SNAPSHOT",
+      "lastUpdatedEpoch": 0,
+      "startTimeEpoch": 1546922023607,
+      "endTimeEpoch": 1546922038745
+    }
+  ]
+}, {
   "id" : "application_1538416563558_0014",
   "name" : "PythonBisectingKMeansExample",
   "attempts" : [ {
@@ -28,19 +46,4 @@
     "startTimeEpoch" : 1524182082734,
     "endTimeEpoch" : 1524182189134
   } ]
-}, {
-  "id" : "application_1516285256255_0012",
-  "name" : "Spark shell",
-  "attempts" : [ {
-    "startTime" : "2018-01-18T18:30:35.119GMT",
-    "endTime" : "2018-01-18T18:38:27.938GMT",
-    "lastUpdated" : "",
-    "duration" : 472819,
-    "sparkUser" : "attilapiros",
-    "completed" : true,
-    "appSparkVersion" : "2.3.0-SNAPSHOT",
-    "lastUpdatedEpoch" : 0,
-    "startTimeEpoch" : 1516300235119,
-    "endTimeEpoch" : 1516300707938
-  } ]
 } ]
diff --git a/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json
index ea9dc1b..ba834d5 100644
--- a/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json
@@ -1,4 +1,22 @@
 [ {
+  "id": "application_1536831636016_59384",
+  "name": "Spark Pi",
+  "attempts": [
+    {
+      "attemptId": "1",
+      "startTime": "2019-01-08T04:33:43.607GMT",
+      "endTime": "2019-01-08T04:33:58.745GMT",
+      "lastUpdated": "",
+      "duration": 15138,
+      "sparkUser": "lajin",
+      "completed": true,
+      "appSparkVersion": "3.0.0-SNAPSHOT",
+      "lastUpdatedEpoch": 0,
+      "startTimeEpoch": 1546922023607,
+      "endTimeEpoch": 1546922038745
+    }
+  ]
+}, {
   "id" : "application_1538416563558_0014",
   "name" : "PythonBisectingKMeansExample",
   "attempts" : [ {
diff --git a/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json
index 2a77071..62d3544 100644
--- a/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json
@@ -1,4 +1,22 @@
 [ {
+  "id": "application_1536831636016_59384",
+  "name": "Spark Pi",
+  "attempts": [
+    {
+      "attemptId": "1",
+      "startTime": "2019-01-08T04:33:43.607GMT",
+      "endTime": "2019-01-08T04:33:58.745GMT",
+      "lastUpdated": "",
+      "duration": 15138,
+      "sparkUser": "lajin",
+      "completed": true,
+      "appSparkVersion": "3.0.0-SNAPSHOT",
+      "lastUpdatedEpoch": 0,
+      "startTimeEpoch": 1546922023607,
+      "endTimeEpoch": 1546922038745
+    }
+  ]
+}, {
   "id" : "application_1538416563558_0014",
   "name" : "PythonBisectingKMeansExample",
   "attempts" : [ {
diff --git a/core/src/test/resources/spark-events/application_1536831636016_59384_1 b/core/src/test/resources/spark-events/application_1536831636016_59384_1
new file mode 100755
index 0000000..69924af
--- /dev/null
+++ b/core/src/test/resources/spark-events/application_1536831636016_59384_1
@@ -0,0 +1,214 @@
+{"Event":"SparkListenerLogStart","Spark Version":"3.0.0-SNAPSHOT"}
+{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"047.company.com","Port":42509},"Maximum Memory":100977868,"Timestamp":1546922024502,"Maximum Onheap Memory":100977868,"Maximum Offheap Memory":0}
+{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/apache/releases/jdk1.8.0_121/jre","Java Version":"1.8.0_121 (Oracle Corporation)","Scala Version":"version 2.12.8"},"Spark Properties":{"spark.history.kerberos.keytab":"/etc/security/keytabs/spark.service.keytab","spark.executor.extraJavaOptions":"-XX:MaxMetaspaceSize=512m -XX:ParallelGCThreads=3","spark.driver.host":"047.company.com","spark.history.fs.logDirectory":"hdfs://hercules-sub/spark-logs","spark.eventLog [...]
+{"Event":"SparkListenerApplicationStart","App Name":"Spark Pi","App ID":"application_1536831636016_59384","Timestamp":1546922023607,"User":"lajin","App Attempt ID":"1","Driver Logs":{"stdout":"https://047.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000001/lajin/stdout?start=-4096","stderr":"https://047.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000001/lajin/stderr?start=-4096"}}
+{"Event":"SparkListenerExecutorAdded","Timestamp":1546922034270,"Executor ID":"2","Executor Info":{"Host":"028.company.com","Total Cores":4,"Log Urls":{"stdout":"https://028.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000003/lajin/stdout?start=-4096","stderr":"https://028.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000003/lajin/stderr?start=-4096"}}}
+{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"2","Host":"028.company.com","Port":46325},"Maximum Memory":97832140,"Timestamp":1546922034365,"Maximum Onheap Memory":97832140,"Maximum Offheap Memory":0}
+{"Event":"SparkListenerExecutorAdded","Timestamp":1546922035929,"Executor ID":"1","Executor Info":{"Host":"036.company.com","Total Cores":4,"Log Urls":{"stdout":"https://036.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000002/lajin/stdout?start=-4096","stderr":"https://036.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000002/lajin/stderr?start=-4096"}}}
+{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"036.company.com","Port":35126},"Maximum Memory":97832140,"Timestamp":1546922036120,"Maximum Onheap Memory":97832140,"Maximum Offheap Memory":0}
+{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1546922036359,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"reduce at SparkPi.scala:38","Number of Tasks":100,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"map\"}","Callsite":"map at SparkPi.scala:34","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":100,"Number of Cached Partitions":0,"Memor [...]
+{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"reduce at SparkPi.scala:38","Number of Tasks":100,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"map\"}","Callsite":"map at SparkPi.scala:34","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":100,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"N [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1546922036494,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1546922036515,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1546922036515,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1546922036515,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1546922036516,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1546922036516,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1546922036516,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1546922036517,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1546922037414,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1546922037416,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1546922036516,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037421,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":86,"Value":86,"Internal":true,"Count Failed Va [...]
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1546922036517,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037424,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":86,"Value":172,"Internal":true,"Count Failed V [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":10,"Attempt":0,"Launch Time":1546922037431,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1546922036515,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037432,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":93,"Value":265,"Internal":true,"Count Failed V [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":11,"Attempt":0,"Launch Time":1546922037432,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1546922036515,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037433,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":93,"Value":358,"Internal":true,"Count Failed V [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":12,"Attempt":0,"Launch Time":1546922037499,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1546922037414,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037500,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":7,"Value":365,"Internal":true,"Count Failed Va [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":13,"Attempt":0,"Launch Time":1546922037500,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1546922037416,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037501,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":7,"Value":372,"Internal":true,"Count Failed Va [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":14,"Attempt":0,"Launch Time":1546922037546,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":12,"Attempt":0,"Launch Time":1546922037499,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037547,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":5795,"Internal":true,"Count Fai [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":15,"Attempt":0,"Launch Time":1546922037583,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1546922036516,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037584,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":280,"Value":652,"Internal":true,"Count Failed  [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":16,"Attempt":0,"Launch Time":1546922037587,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":14,"Attempt":0,"Launch Time":1546922037546,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037588,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":7420,"Internal":true,"Count Fai [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":17,"Attempt":0,"Launch Time":1546922037624,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":16,"Attempt":0,"Launch Time":1546922037587,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037624,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":8211,"Internal":true,"Count Fai [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":18,"Attempt":0,"Launch Time":1546922037633,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1546922036494,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037634,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":286,"Value":938,"Internal":true,"Count Failed  [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":19,"Attempt":0,"Launch Time":1546922037639,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":15,"Attempt":0,"Launch Time":1546922037583,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037639,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":944,"Internal":true,"Count Failed  [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":20,"Attempt":0,"Launch Time":1546922037641,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1546922036515,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037641,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":286,"Value":1230,"Internal":true,"Count Failed [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":21,"Attempt":0,"Launch Time":1546922037642,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1546922036516,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037642,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":286,"Value":1516,"Internal":true,"Count Failed [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":22,"Attempt":0,"Launch Time":1546922037662,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":17,"Attempt":0,"Launch Time":1546922037624,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037662,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":12338,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":23,"Attempt":0,"Launch Time":1546922037665,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":10,"Attempt":0,"Launch Time":1546922037431,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037665,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":13129,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":24,"Attempt":0,"Launch Time":1546922037671,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":11,"Attempt":0,"Launch Time":1546922037432,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037672,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":13920,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":25,"Attempt":0,"Launch Time":1546922037685,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":21,"Attempt":0,"Launch Time":1546922037642,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037685,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":14711,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":26,"Attempt":0,"Launch Time":1546922037699,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":22,"Attempt":0,"Launch Time":1546922037662,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037700,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":15502,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":27,"Attempt":0,"Launch Time":1546922037724,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":25,"Attempt":0,"Launch Time":1546922037685,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037724,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":16293,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":28,"Attempt":0,"Launch Time":1546922037740,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":26,"Attempt":0,"Launch Time":1546922037699,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037740,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":17084,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":29,"Attempt":0,"Launch Time":1546922037744,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":18,"Attempt":0,"Launch Time":1546922037633,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037745,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":17875,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":30,"Attempt":0,"Launch Time":1546922037758,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":27,"Attempt":0,"Launch Time":1546922037724,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037759,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":18666,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":31,"Attempt":0,"Launch Time":1546922037779,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":28,"Attempt":0,"Launch Time":1546922037740,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037780,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":19457,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":32,"Index":32,"Attempt":0,"Launch Time":1546922037781,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":13,"Attempt":0,"Launch Time":1546922037500,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037781,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":20248,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":33,"Index":33,"Attempt":0,"Launch Time":1546922037791,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":30,"Attempt":0,"Launch Time":1546922037758,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037792,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":20996,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":34,"Index":34,"Attempt":0,"Launch Time":1546922037822,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":32,"Index":32,"Attempt":0,"Launch Time":1546922037781,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037822,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"C [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":35,"Index":35,"Attempt":0,"Launch Time":1546922037825,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":33,"Index":33,"Attempt":0,"Launch Time":1546922037791,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037825,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":22621,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":36,"Index":36,"Attempt":0,"Launch Time":1546922037857,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":35,"Index":35,"Attempt":0,"Launch Time":1546922037825,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037858,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":23412,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":37,"Index":37,"Attempt":0,"Launch Time":1546922037865,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":34,"Index":34,"Attempt":0,"Launch Time":1546922037822,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037866,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":24203,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":38,"Index":38,"Attempt":0,"Launch Time":1546922037890,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":36,"Index":36,"Attempt":0,"Launch Time":1546922037857,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037890,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":24994,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":39,"Index":39,"Attempt":0,"Launch Time":1546922037894,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":19,"Attempt":0,"Launch Time":1546922037639,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037894,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":25785,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":40,"Index":40,"Attempt":0,"Launch Time":1546922037917,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":37,"Index":37,"Attempt":0,"Launch Time":1546922037865,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037917,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1522,"Internal":true,"Count Failed [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":41,"Index":41,"Attempt":0,"Launch Time":1546922037919,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":23,"Attempt":0,"Launch Time":1546922037665,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037919,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1528,"Internal":true,"Count Failed [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":42,"Index":42,"Attempt":0,"Launch Time":1546922037922,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":38,"Index":38,"Attempt":0,"Launch Time":1546922037890,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037923,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":28201,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":43,"Index":43,"Attempt":0,"Launch Time":1546922037949,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":24,"Attempt":0,"Launch Time":1546922037671,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037952,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1534,"Internal":true,"Count Failed [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":44,"Index":44,"Attempt":0,"Launch Time":1546922037952,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":20,"Attempt":0,"Launch Time":1546922037641,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037953,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":29826,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":45,"Index":45,"Attempt":0,"Launch Time":1546922037953,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":29,"Attempt":0,"Launch Time":1546922037744,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037953,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":30617,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":46,"Index":46,"Attempt":0,"Launch Time":1546922037954,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":39,"Index":39,"Attempt":0,"Launch Time":1546922037894,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037954,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":31408,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":47,"Index":47,"Attempt":0,"Launch Time":1546922037955,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":41,"Index":41,"Attempt":0,"Launch Time":1546922037919,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037955,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":2,"Internal":true,"C [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":48,"Index":48,"Attempt":0,"Launch Time":1546922037990,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":47,"Index":47,"Attempt":0,"Launch Time":1546922037955,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037990,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":33033,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":49,"Index":49,"Attempt":0,"Launch Time":1546922037991,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":46,"Index":46,"Attempt":0,"Launch Time":1546922037954,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037992,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":33781,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":50,"Index":50,"Attempt":0,"Launch Time":1546922038023,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":49,"Index":49,"Attempt":0,"Launch Time":1546922037991,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038024,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":34529,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":51,"Index":51,"Attempt":0,"Launch Time":1546922038025,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":48,"Index":48,"Attempt":0,"Launch Time":1546922037990,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038025,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":35320,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":52,"Index":52,"Attempt":0,"Launch Time":1546922038060,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":51,"Index":51,"Attempt":0,"Launch Time":1546922038025,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038061,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":36111,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":53,"Index":53,"Attempt":0,"Launch Time":1546922038063,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":50,"Index":50,"Attempt":0,"Launch Time":1546922038023,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038064,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":3,"Internal":true,"C [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":54,"Index":54,"Attempt":0,"Launch Time":1546922038095,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":52,"Index":52,"Attempt":0,"Launch Time":1546922038060,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038095,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":37779,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":55,"Index":55,"Attempt":0,"Launch Time":1546922038097,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":53,"Index":53,"Attempt":0,"Launch Time":1546922038063,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038097,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":38570,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":56,"Index":56,"Attempt":0,"Launch Time":1546922038113,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":31,"Index":31,"Attempt":0,"Launch Time":1546922037779,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038113,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1546,"Internal":true,"Count Failed [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":57,"Index":57,"Attempt":0,"Launch Time":1546922038131,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":54,"Index":54,"Attempt":0,"Launch Time":1546922038095,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038131,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":40195,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":58,"Index":58,"Attempt":0,"Launch Time":1546922038137,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":44,"Index":44,"Attempt":0,"Launch Time":1546922037952,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038137,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1552,"Internal":true,"Count Failed [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":59,"Index":59,"Attempt":0,"Launch Time":1546922038148,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":55,"Index":55,"Attempt":0,"Launch Time":1546922038097,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038148,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":41820,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":60,"Index":60,"Attempt":0,"Launch Time":1546922038168,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":57,"Index":57,"Attempt":0,"Launch Time":1546922038131,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038169,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":42611,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":61,"Index":61,"Attempt":0,"Launch Time":1546922038175,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":40,"Index":40,"Attempt":0,"Launch Time":1546922037917,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038175,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":43402,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":62,"Index":62,"Attempt":0,"Launch Time":1546922038185,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":59,"Index":59,"Attempt":0,"Launch Time":1546922038148,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038185,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":44193,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":63,"Index":63,"Attempt":0,"Launch Time":1546922038204,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":60,"Index":60,"Attempt":0,"Launch Time":1546922038168,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038204,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":44941,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":64,"Index":64,"Attempt":0,"Launch Time":1546922038217,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":42,"Index":42,"Attempt":0,"Launch Time":1546922037922,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038217,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1558,"Internal":true,"Count Failed [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":65,"Index":65,"Attempt":0,"Launch Time":1546922038218,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":62,"Index":62,"Attempt":0,"Launch Time":1546922038185,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038218,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":46523,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":66,"Index":66,"Attempt":0,"Launch Time":1546922038220,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":45,"Index":45,"Attempt":0,"Launch Time":1546922037953,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038221,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1564,"Internal":true,"Count Failed [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":67,"Index":67,"Attempt":0,"Launch Time":1546922038239,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":63,"Index":63,"Attempt":0,"Launch Time":1546922038204,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038239,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":4,"Internal":true,"C [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":68,"Index":68,"Attempt":0,"Launch Time":1546922038251,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":64,"Index":64,"Attempt":0,"Launch Time":1546922038217,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038252,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":5,"Internal":true,"C [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":69,"Index":69,"Attempt":0,"Launch Time":1546922038253,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":43,"Index":43,"Attempt":0,"Launch Time":1546922037949,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038253,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":49773,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":70,"Index":70,"Attempt":0,"Launch Time":1546922038275,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":67,"Index":67,"Attempt":0,"Launch Time":1546922038239,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038276,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":50521,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":71,"Index":71,"Attempt":0,"Launch Time":1546922038291,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":68,"Index":68,"Attempt":0,"Launch Time":1546922038251,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038292,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":51269,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":72,"Index":72,"Attempt":0,"Launch Time":1546922038294,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":58,"Index":58,"Attempt":0,"Launch Time":1546922038137,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038295,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":52060,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":73,"Index":73,"Attempt":0,"Launch Time":1546922038313,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":70,"Index":70,"Attempt":0,"Launch Time":1546922038275,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038313,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":52808,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":74,"Index":74,"Attempt":0,"Launch Time":1546922038329,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":71,"Index":71,"Attempt":0,"Launch Time":1546922038291,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038330,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":53599,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":75,"Index":75,"Attempt":0,"Launch Time":1546922038347,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":73,"Index":73,"Attempt":0,"Launch Time":1546922038313,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038348,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":54390,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":76,"Index":76,"Attempt":0,"Launch Time":1546922038368,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":74,"Index":74,"Attempt":0,"Launch Time":1546922038329,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038368,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":55181,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":77,"Index":77,"Attempt":0,"Launch Time":1546922038390,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":75,"Index":75,"Attempt":0,"Launch Time":1546922038347,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038391,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1570,"Internal":true,"Count Failed [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":78,"Index":78,"Attempt":0,"Launch Time":1546922038409,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":76,"Index":76,"Attempt":0,"Launch Time":1546922038368,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038409,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":56806,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":79,"Index":79,"Attempt":0,"Launch Time":1546922038416,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":61,"Index":61,"Attempt":0,"Launch Time":1546922038175,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038416,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1576,"Internal":true,"Count Failed [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":80,"Index":80,"Attempt":0,"Launch Time":1546922038424,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":56,"Index":56,"Attempt":0,"Launch Time":1546922038113,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038424,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1582,"Internal":true,"Count Failed [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":81,"Index":81,"Attempt":0,"Launch Time":1546922038436,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":69,"Index":69,"Attempt":0,"Launch Time":1546922038253,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038436,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1588,"Internal":true,"Count Failed [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":82,"Index":82,"Attempt":0,"Launch Time":1546922038448,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":78,"Index":78,"Attempt":0,"Launch Time":1546922038409,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038448,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":60099,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":83,"Index":83,"Attempt":0,"Launch Time":1546922038483,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":65,"Index":65,"Attempt":0,"Launch Time":1546922038218,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038483,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1594,"Internal":true,"Count Failed [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":84,"Index":84,"Attempt":0,"Launch Time":1546922038492,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":82,"Index":82,"Attempt":0,"Launch Time":1546922038448,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038493,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1600,"Internal":true,"Count Failed [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":85,"Index":85,"Attempt":0,"Launch Time":1546922038495,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":66,"Index":66,"Attempt":0,"Launch Time":1546922038220,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038495,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1606,"Internal":true,"Count Failed [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":86,"Index":86,"Attempt":0,"Launch Time":1546922038507,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":77,"Index":77,"Attempt":0,"Launch Time":1546922038390,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038507,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":63263,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":87,"Index":87,"Attempt":0,"Launch Time":1546922038517,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":80,"Index":80,"Attempt":0,"Launch Time":1546922038424,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038518,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":64054,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":88,"Index":88,"Attempt":0,"Launch Time":1546922038533,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":84,"Index":84,"Attempt":0,"Launch Time":1546922038492,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038533,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":64802,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":89,"Index":89,"Attempt":0,"Launch Time":1546922038536,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":81,"Index":81,"Attempt":0,"Launch Time":1546922038436,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038537,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":65593,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":90,"Index":90,"Attempt":0,"Launch Time":1546922038550,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":72,"Index":72,"Attempt":0,"Launch Time":1546922038294,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038550,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1612,"Internal":true,"Count Failed [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":91,"Index":91,"Attempt":0,"Launch Time":1546922038551,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":79,"Index":79,"Attempt":0,"Launch Time":1546922038416,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038552,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":67175,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":92,"Index":92,"Attempt":0,"Launch Time":1546922038574,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":88,"Index":88,"Attempt":0,"Launch Time":1546922038533,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038574,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":67966,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":93,"Index":93,"Attempt":0,"Launch Time":1546922038612,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":86,"Index":86,"Attempt":0,"Launch Time":1546922038507,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038612,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":68757,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":94,"Index":94,"Attempt":0,"Launch Time":1546922038614,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":85,"Index":85,"Attempt":0,"Launch Time":1546922038495,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038614,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":6,"Internal":true,"C [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":95,"Index":95,"Attempt":0,"Launch Time":1546922038621,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":87,"Index":87,"Attempt":0,"Launch Time":1546922038517,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038621,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":70339,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":96,"Index":96,"Attempt":0,"Launch Time":1546922038636,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":89,"Index":89,"Attempt":0,"Launch Time":1546922038536,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038637,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":7,"Internal":true,"C [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":97,"Index":97,"Attempt":0,"Launch Time":1546922038640,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":92,"Index":92,"Attempt":0,"Launch Time":1546922038574,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038641,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":71921,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":98,"Index":98,"Attempt":0,"Launch Time":1546922038664,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":91,"Index":91,"Attempt":0,"Launch Time":1546922038551,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038664,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":72669,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":99,"Index":99,"Attempt":0,"Launch Time":1546922038680,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":94,"Index":94,"Attempt":0,"Launch Time":1546922038614,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038681,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":73460,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":83,"Index":83,"Attempt":0,"Launch Time":1546922038483,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038705,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":74251,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":97,"Index":97,"Attempt":0,"Launch Time":1546922038640,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038705,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":75042,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":93,"Index":93,"Attempt":0,"Launch Time":1546922038612,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038717,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":75833,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":99,"Index":99,"Attempt":0,"Launch Time":1546922038680,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038724,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":76624,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":90,"Index":90,"Attempt":0,"Launch Time":1546922038550,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038725,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":77415,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":95,"Index":95,"Attempt":0,"Launch Time":1546922038621,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038726,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":78206,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":96,"Index":96,"Attempt":0,"Launch Time":1546922038636,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038729,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":78954,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":98,"Index":98,"Attempt":0,"Launch Time":1546922038664,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038733,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":79745,"Internal":true,"Count Fa [...]
+{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":211171816,"JVMOffHeapMemory":90237256,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":4876,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":4876,"OffHeapUnifiedMemory":0,"DirectPoolMemory":806275,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":2646888448,"ProcessTreeJVMRSSMemory":520900608,"ProcessTreePythonVMemory":0,"ProcessTree [...]
+{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"reduce at SparkPi.scala:38","Number of Tasks":100,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"map\"}","Callsite":"map at SparkPi.scala:34","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":100,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"N [...]
+{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1546922038738,"Job Result":{"Result":"JobSucceeded"}}
+{"Event":"SparkListenerApplicationEnd","Timestamp":1546922038745}
diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
index 1a071fa..6665a89 100644
--- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
@@ -137,6 +137,8 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers
       "applications/application_1506645932520_24630151/executors",
     "executor list with executor process tree metrics json" ->
       "applications/application_1538416563558_0014/executors",
+    "executor list with executor garbage collection metrics json" ->
+      "applications/application_1536831636016_59384/1/executors",
     "stage list json" -> "applications/local-1422981780767/stages",
     "complete stage list json" -> "applications/local-1422981780767/stages?status=complete",
     "failed stage list json" -> "applications/local-1422981780767/stages?status=failed",
diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala
index 40521f0..350fc2a 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala
@@ -285,66 +285,66 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit
       // with different peak updates for each executor
       createExecutorMetricsUpdateEvent(1,
         new ExecutorMetrics(Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L, 7500L, 3500L,
-          6500L, 2500L, 5500L, 1500L))),
+          6500L, 2500L, 5500L, 1500L, 10L, 90L, 2L, 20L))),
       createExecutorMetricsUpdateEvent(2,
         new ExecutorMetrics(Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L, 8500L, 3500L,
-          7500L, 2500L, 6500L, 1500L))),
+          7500L, 2500L, 6500L, 1500L, 10L, 90L, 2L, 20L))),
       // exec 1: new stage 0 peaks for metrics at indexes:  2, 4, 6
       createExecutorMetricsUpdateEvent(1,
         new ExecutorMetrics(Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L, 8000L, 4000L,
-          7000L, 3000L, 6000L, 2000L))),
+          7000L, 3000L, 6000L, 2000L, 10L, 90L, 2L, 20L))),
       // exec 2: new stage 0 peaks for metrics at indexes: 0, 4, 6
       createExecutorMetricsUpdateEvent(2,
         new ExecutorMetrics(Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L, 9000L, 4000L,
-          8000L, 3000L, 7000L, 2000L))),
+          8000L, 3000L, 7000L, 2000L, 10L, 90L, 2L, 20L))),
       // exec 1: new stage 0 peaks for metrics at indexes: 5, 7
       createExecutorMetricsUpdateEvent(1,
         new ExecutorMetrics(Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L, 8000L, 3500L,
-          7000L, 2500L, 6000L, 1500L))),
+          7000L, 2500L, 6000L, 1500L, 10L, 90L, 2L, 20L))),
       // exec 2: new stage 0 peaks for metrics at indexes: 0, 5, 6, 7, 8
       createExecutorMetricsUpdateEvent(2,
         new ExecutorMetrics(Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L, 8500L, 3500L,
-          7500L, 2500L, 6500L, 1500L))),
+          7500L, 2500L, 6500L, 1500L, 10L, 90L, 2L, 20L))),
       // now start stage 1, one more metric update for each executor, and new
       // peaks for some stage 1 metrics (as listed), initialize stage 1 peaks
       createStageSubmittedEvent(1),
       // exec 1: new stage 0 peaks for metrics at indexes: 0, 3, 7; initialize stage 1 peaks
       createExecutorMetricsUpdateEvent(1,
         new ExecutorMetrics(Array(5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L,
-          0L, 5000L, 3000L, 4000L, 2000L, 3000L, 1000L))),
+          0L, 5000L, 3000L, 4000L, 2000L, 3000L, 1000L, 10L, 90L, 2L, 20L))),
       // exec 2: new stage 0 peaks for metrics at indexes: 0, 1, 3, 6, 7, 9;
       // initialize stage 1 peaks
       createExecutorMetricsUpdateEvent(2,
         new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 0L, 10L, 50L, 30L, 10L,
-          40L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L))),
+          40L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L, 10L, 90L, 2L, 20L))),
       // complete stage 0, and 3 more updates for each executor with just
       // stage 1 running
       createStageCompletedEvent(0),
       // exec 1: new stage 1 peaks for metrics at indexes: 0, 1, 3
       createExecutorMetricsUpdateEvent(1,
         new ExecutorMetrics(Array(6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L, 5000L, 3000L,
-          4000L, 2000L, 3000L, 1000L))),
+          4000L, 2000L, 3000L, 1000L, 10L, 90L, 2L, 20L))),
       // exec 2: new stage 1 peaks for metrics at indexes: 3, 4, 7, 8
       createExecutorMetricsUpdateEvent(2,
         new ExecutorMetrics(Array(5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L,
-          20L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L, 5000L, 2000L))),
+          20L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L, 10L, 90L, 2L, 20L))),
       // exec 1: new stage 1 peaks for metrics at indexes: 0, 4, 5, 7
       createExecutorMetricsUpdateEvent(1,
         new ExecutorMetrics(Array(7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L, 3000L, 2500L,
-          2000L, 1500L, 1000L, 500L))),
+          2000L, 1500L, 1000L, 500L, 10L, 90L, 2L, 20L))),
       // exec 2: new stage 1 peak for metrics at index: 7
       createExecutorMetricsUpdateEvent(2,
         new ExecutorMetrics(Array(5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L,
-          20L, 7000L, 3000L, 6000L, 2000L, 5000L, 1000L))),
+          20L, 7000L, 3000L, 6000L, 2000L, 5000L, 1000L, 10L, 90L, 2L, 20L))),
       // exec 1: no new stage 1 peaks
       createExecutorMetricsUpdateEvent(1,
         new ExecutorMetrics(Array(5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L,
-          0L, 4000L, 2500L, 3000L, 1500L, 2000L, 500L))),
+          0L, 4000L, 2500L, 3000L, 1500L, 2000L, 500L, 10L, 90L, 2L, 20L))),
       createExecutorRemovedEvent(1),
       // exec 2: new stage 1 peak for metrics at index: 6
       createExecutorMetricsUpdateEvent(2,
         new ExecutorMetrics(Array(4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L, 7000L,
-          4000L, 6000L, 3000L, 5000L, 2000L))),
+          4000L, 6000L, 3000L, 5000L, 2000L, 10L, 90L, 2L, 20L))),
       createStageCompletedEvent(1),
       SparkListenerApplicationEnd(1000L))
 
@@ -362,19 +362,19 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit
       ((0, "1"),
         new SparkListenerStageExecutorMetrics("1", 0, 0,
           new ExecutorMetrics(Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L,
-            70L, 20L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L)))),
+            70L, 20L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L, 10L, 90L, 2L, 20L)))),
       ((0, "2"),
         new SparkListenerStageExecutorMetrics("2", 0, 0,
           new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L,
-            80L, 40L, 9000L, 4000L, 8000L, 3000L, 7000L, 2000L)))),
+            80L, 40L, 9000L, 4000L, 8000L, 3000L, 7000L, 2000L, 10L, 90L, 2L, 20L)))),
       ((1, "1"),
         new SparkListenerStageExecutorMetrics("1", 1, 0,
           new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L,
-            50L, 0L, 5000L, 3000L, 4000L, 2000L, 3000L, 1000L)))),
+            50L, 0L, 5000L, 3000L, 4000L, 2000L, 3000L, 1000L, 10L, 90L, 2L, 20L)))),
       ((1, "2"),
         new SparkListenerStageExecutorMetrics("2", 1, 0,
           new ExecutorMetrics(Array(7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L,
-            40L, 40L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L)))))
+            40L, 40L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L, 10L, 90L, 2L, 20L)))))
     // Verify the log file contains the expected events.
     // Posted events should be logged, except for ExecutorMetricsUpdate events -- these
     // are consolidated, and the peak values for each stage are logged at stage end.
diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala
index c3ff379..f2f62d6 100644
--- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala
@@ -99,7 +99,7 @@ class JsonProtocolSuite extends SparkFunSuite {
           .zipWithIndex.map { case (a, i) => a.copy(id = i) }
       val executorUpdates = new ExecutorMetrics(
         Array(543L, 123456L, 12345L, 1234L, 123L, 12L, 432L,
-          321L, 654L, 765L, 256912L, 123456L, 123456L, 61728L, 30364L, 15182L))
+          321L, 654L, 765L, 256912L, 123456L, 123456L, 61728L, 30364L, 15182L, 10L, 90L, 2L, 20L))
       SparkListenerExecutorMetricsUpdate("exec3", Seq((1L, 2, 3, accumUpdates)),
         Some(executorUpdates))
     }
@@ -109,7 +109,7 @@ class JsonProtocolSuite extends SparkFunSuite {
     val stageExecutorMetrics =
       SparkListenerStageExecutorMetrics("1", 2, 3,
         new ExecutorMetrics(Array(543L, 123456L, 12345L, 1234L, 123L, 12L, 432L,
-          321L, 654L, 765L, 256912L, 123456L, 123456L, 61728L, 30364L, 15182L)))
+          321L, 654L, 765L, 256912L, 123456L, 123456L, 61728L, 30364L, 15182L, 10L, 90L, 2L, 20L)))
     testEvent(stageSubmitted, stageSubmittedJsonString)
     testEvent(stageCompleted, stageCompletedJsonString)
     testEvent(taskStart, taskStartJsonString)
@@ -888,7 +888,7 @@ private[spark] object JsonProtocolSuite extends Assertions {
     val executorMetricsUpdate =
       if (includeExecutorMetrics) {
         Some(new ExecutorMetrics(Array(123456L, 543L, 0L, 0L, 0L, 0L, 0L,
-          0L, 0L, 0L, 256912L, 123456L, 123456L, 61728L, 30364L, 15182L)))
+          0L, 0L, 0L, 256912L, 123456L, 123456L, 61728L, 30364L, 15182L, 10L, 90L, 2L, 20L)))
       } else {
         None
       }
@@ -2106,7 +2106,11 @@ private[spark] object JsonProtocolSuite extends Assertions {
       |    "ProcessTreePythonVMemory": 123456,
       |    "ProcessTreePythonRSSMemory": 61728,
       |    "ProcessTreeOtherVMemory": 30364,
-      |    "ProcessTreeOtherRSSMemory": 15182
+      |    "ProcessTreeOtherRSSMemory": 15182,
+      |    "MinorGCCount": 10,
+      |    "MinorGCTime": 90,
+      |    "MajorGCCount": 2,
+      |    "MajorGCTime": 20
       |  }
       |
       |}
@@ -2135,7 +2139,11 @@ private[spark] object JsonProtocolSuite extends Assertions {
       |    "ProcessTreePythonVMemory": 123456,
       |    "ProcessTreePythonRSSMemory": 61728,
       |    "ProcessTreeOtherVMemory": 30364,
-      |    "ProcessTreeOtherRSSMemory": 15182
+      |    "ProcessTreeOtherRSSMemory": 15182,
+      |    "MinorGCCount": 10,
+      |    "MinorGCTime": 90,
+      |    "MajorGCCount": 2,
+      |    "MajorGCTime": 20
       |  }
       |}
     """.stripMargin
diff --git a/dev/.rat-excludes b/dev/.rat-excludes
index 8239cbc..59e619b 100644
--- a/dev/.rat-excludes
+++ b/dev/.rat-excludes
@@ -83,6 +83,7 @@ app-20161116163331-0000
 application_1516285256255_0012
 application_1506645932520_24630151
 application_1538416563558_0014
+application_1536831636016_59384_1
 stat
 local-1422981759269
 local-1422981780767


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