[SPARK-23431][CORE] Expose stage level peak executor metrics via REST API

### What changes were proposed in this pull request?

Note that this PR is forked from #23340 originally written by edwinalu.

This PR proposes to expose the peak executor metrics at the stage level via the REST APIs:
* `/applications/<application_id>/stages/`: peak values of executor metrics for **each stage**
* `/applications/<application_id>/stages/<stage_id>/< stage_attempt_id >`: peak values of executor metrics for **each executor** for the stage, followed by peak values of executor metrics for the stage

### Why are the changes needed?

The stage level peak executor metrics can help better understand your application's resource utilization.

### Does this PR introduce _any_ user-facing change?

1. For the `/applications/<application_id>/stages/` API, you will see the following new info for **each stage**:
```JSON
  "peakExecutorMetrics" : {
    "JVMHeapMemory" : 213367864,
    "JVMOffHeapMemory" : 189011656,
    "OnHeapExecutionMemory" : 0,
    "OffHeapExecutionMemory" : 0,
    "OnHeapStorageMemory" : 2133349,
    "OffHeapStorageMemory" : 0,
    "OnHeapUnifiedMemory" : 2133349,
    "OffHeapUnifiedMemory" : 0,
    "DirectPoolMemory" : 282024,
    "MappedPoolMemory" : 0,
    "ProcessTreeJVMVMemory" : 0,
    "ProcessTreeJVMRSSMemory" : 0,
    "ProcessTreePythonVMemory" : 0,
    "ProcessTreePythonRSSMemory" : 0,
    "ProcessTreeOtherVMemory" : 0,
    "ProcessTreeOtherRSSMemory" : 0,
    "MinorGCCount" : 13,
    "MinorGCTime" : 115,
    "MajorGCCount" : 4,
    "MajorGCTime" : 339
  }
```

2. For the `/applications/<application_id>/stages/<stage_id>/<stage_attempt_id>` API, you will see the following new info for **each executor** under `executorSummary`:
```JSON
  "peakMemoryMetrics" : {
    "JVMHeapMemory" : 0,
    "JVMOffHeapMemory" : 0,
    "OnHeapExecutionMemory" : 0,
    "OffHeapExecutionMemory" : 0,
    "OnHeapStorageMemory" : 0,
    "OffHeapStorageMemory" : 0,
    "OnHeapUnifiedMemory" : 0,
    "OffHeapUnifiedMemory" : 0,
    "DirectPoolMemory" : 0,
    "MappedPoolMemory" : 0,
    "ProcessTreeJVMVMemory" : 0,
    "ProcessTreeJVMRSSMemory" : 0,
    "ProcessTreePythonVMemory" : 0,
    "ProcessTreePythonRSSMemory" : 0,
    "ProcessTreeOtherVMemory" : 0,
    "ProcessTreeOtherRSSMemory" : 0,
    "MinorGCCount" : 0,
    "MinorGCTime" : 0,
    "MajorGCCount" : 0,
    "MajorGCTime" : 0
  }
```
, and the following at the stage level:
```JSON
"peakExecutorMetrics" : {
    "JVMHeapMemory" : 213367864,
    "JVMOffHeapMemory" : 189011656,
    "OnHeapExecutionMemory" : 0,
    "OffHeapExecutionMemory" : 0,
    "OnHeapStorageMemory" : 2133349,
    "OffHeapStorageMemory" : 0,
    "OnHeapUnifiedMemory" : 2133349,
    "OffHeapUnifiedMemory" : 0,
    "DirectPoolMemory" : 282024,
    "MappedPoolMemory" : 0,
    "ProcessTreeJVMVMemory" : 0,
    "ProcessTreeJVMRSSMemory" : 0,
    "ProcessTreePythonVMemory" : 0,
    "ProcessTreePythonRSSMemory" : 0,
    "ProcessTreeOtherVMemory" : 0,
    "ProcessTreeOtherRSSMemory" : 0,
    "MinorGCCount" : 13,
    "MinorGCTime" : 115,
    "MajorGCCount" : 4,
    "MajorGCTime" : 339
  }
```

### How was this patch tested?

Added tests.

Closes #29020 from imback82/metrics.

Lead-authored-by: Terry Kim <yuminkim@gmail.com>
Co-authored-by: edwinalu <edwina.lu@gmail.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala
index f7b0e9b..7ae9117 100644
--- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala
+++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala
@@ -24,7 +24,7 @@
 import scala.collection.mutable.HashMap
 
 import org.apache.spark._
-import org.apache.spark.executor.TaskMetrics
+import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics}
 import org.apache.spark.internal.Logging
 import org.apache.spark.internal.config.CPUS_PER_TASK
 import org.apache.spark.internal.config.Status._
@@ -868,13 +868,17 @@
     // check if there is a new peak value for any of the executor level memory metrics
     // for the live UI. SparkListenerExecutorMetricsUpdate events are only processed
     // for the live UI.
-    event.executorUpdates.foreach { case (_, peakUpdates) =>
+    event.executorUpdates.foreach { case (key, peakUpdates) =>
       liveExecutors.get(event.execId).foreach { exec =>
         if (exec.peakExecutorMetrics.compareAndUpdatePeakValues(peakUpdates)) {
-          maybeUpdate(exec, now)
+          update(exec, now)
         }
       }
+
+      // Update stage level peak executor metrics.
+      updateStageLevelPeakExecutorMetrics(key._1, key._2, event.execId, peakUpdates, now)
     }
+
     // Flush updates if necessary. Executor heartbeat is an event that happens periodically. Flush
     // here to ensure the staleness of Spark UI doesn't last more than
     // `max(heartbeat interval, liveUpdateMinFlushPeriod)`.
@@ -885,17 +889,38 @@
     }
   }
 
-  override def onStageExecutorMetrics(executorMetrics: SparkListenerStageExecutorMetrics): Unit = {
+  override def onStageExecutorMetrics(event: SparkListenerStageExecutorMetrics): Unit = {
     val now = System.nanoTime()
 
     // check if there is a new peak value for any of the executor level memory metrics,
     // while reading from the log. SparkListenerStageExecutorMetrics are only processed
     // when reading logs.
-    liveExecutors.get(executorMetrics.execId).orElse(
-      deadExecutors.get(executorMetrics.execId)).foreach { exec =>
-        if (exec.peakExecutorMetrics.compareAndUpdatePeakValues(executorMetrics.executorMetrics)) {
-          update(exec, now)
-        }
+    liveExecutors.get(event.execId).orElse(
+      deadExecutors.get(event.execId)).foreach { exec =>
+      if (exec.peakExecutorMetrics.compareAndUpdatePeakValues(event.executorMetrics)) {
+        update(exec, now)
+      }
+    }
+
+    // Update stage level peak executor metrics.
+    updateStageLevelPeakExecutorMetrics(
+      event.stageId, event.stageAttemptId, event.execId, event.executorMetrics, now)
+  }
+
+  private def updateStageLevelPeakExecutorMetrics(
+      stageId: Int,
+      stageAttemptId: Int,
+      executorId: String,
+      executorMetrics: ExecutorMetrics,
+      now: Long): Unit = {
+    Option(liveStages.get((stageId, stageAttemptId))).foreach { stage =>
+      if (stage.peakExecutorMetrics.compareAndUpdatePeakValues(executorMetrics)) {
+        update(stage, now)
+      }
+      val esummary = stage.executorSummary(executorId)
+      if (esummary.peakExecutorMetrics.compareAndUpdatePeakValues(executorMetrics)) {
+        update(esummary, now)
+      }
     }
   }
 
diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala
index 0a8d188..5c6543f 100644
--- a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala
+++ b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala
@@ -506,7 +506,8 @@
       tasks = Some(tasks),
       executorSummary = Some(executorSummary(stage.stageId, stage.attemptId)),
       killedTasksSummary = stage.killedTasksSummary,
-      resourceProfileId = stage.resourceProfileId)
+      resourceProfileId = stage.resourceProfileId,
+      peakExecutorMetrics = stage.peakExecutorMetrics)
   }
 
   def rdd(rddId: Int): v1.RDDStorageInfo = {
diff --git a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala
index 8147821..0fadd33 100644
--- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala
+++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala
@@ -365,6 +365,8 @@
 
   var metrics = createMetrics(default = 0L)
 
+  val peakExecutorMetrics = new ExecutorMetrics()
+
   override protected def doUpdate(): Any = {
     val info = new v1.ExecutorStageSummary(
       taskTime,
@@ -381,7 +383,8 @@
       metrics.shuffleWriteMetrics.recordsWritten,
       metrics.memoryBytesSpilled,
       metrics.diskBytesSpilled,
-      isBlacklisted)
+      isBlacklisted,
+      Some(peakExecutorMetrics).filter(_.isSet))
     new ExecutorStageSummaryWrapper(stageId, attemptId, executorId, info)
   }
 
@@ -420,6 +423,8 @@
 
   var blackListedExecutors = new HashSet[String]()
 
+  val peakExecutorMetrics = new ExecutorMetrics()
+
   // Used for cleanup of tasks after they reach the configured limit. Not written to the store.
   @volatile var cleaning = false
   var savedTasks = new AtomicInteger(0)
@@ -484,7 +489,8 @@
       tasks = None,
       executorSummary = None,
       killedTasksSummary = killedSummary,
-      resourceProfileId = info.resourceProfileId)
+      resourceProfileId = info.resourceProfileId,
+      Some(peakExecutorMetrics).filter(_.isSet))
   }
 
   override protected def doUpdate(): Any = {
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala
index e89e291..d207a60 100644
--- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala
@@ -82,7 +82,10 @@
     val shuffleWriteRecords : Long,
     val memoryBytesSpilled : Long,
     val diskBytesSpilled : Long,
-    val isBlacklistedForStage: Boolean)
+    val isBlacklistedForStage: Boolean,
+    @JsonSerialize(using = classOf[ExecutorMetricsJsonSerializer])
+    @JsonDeserialize(using = classOf[ExecutorMetricsJsonDeserializer])
+    val peakMemoryMetrics: Option[ExecutorMetrics])
 
 class ExecutorSummary private[spark](
     val id: String,
@@ -259,7 +262,10 @@
     val tasks: Option[Map[Long, TaskData]],
     val executorSummary: Option[Map[String, ExecutorStageSummary]],
     val killedTasksSummary: Map[String, Int],
-    val resourceProfileId: Int)
+    val resourceProfileId: Int,
+    @JsonSerialize(using = classOf[ExecutorMetricsJsonSerializer])
+    @JsonDeserialize(using = classOf[ExecutorMetricsJsonDeserializer])
+    val peakExecutorMetrics: Option[ExecutorMetrics])
 
 class TaskData private[spark](
     val taskId: Long,
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala
index bba5e3d..df239d6 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala
@@ -255,7 +255,8 @@
           tasks = None,
           executorSummary = None,
           killedTasksSummary = Map(),
-          ResourceProfile.UNKNOWN_RESOURCE_PROFILE_ID)
+          ResourceProfile.UNKNOWN_RESOURCE_PROFILE_ID,
+          peakExecutorMetrics = None)
       }
     }
 
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 d2b3d1b..06015ec 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,19 @@
 [ {
+  "id" : "app-20200706201101-0003",
+  "name" : "Spark shell",
+  "attempts" : [ {
+    "startTime" : "2020-07-07T03:11:00.235GMT",
+    "endTime" : "2020-07-07T03:17:04.231GMT",
+    "lastUpdated" : "",
+    "duration" : 363996,
+    "sparkUser" : "terryk",
+    "completed" : true,
+    "appSparkVersion" : "3.1.0-SNAPSHOT",
+    "endTimeEpoch" : 1594091824231,
+    "startTimeEpoch" : 1594091460235,
+    "lastUpdatedEpoch" : 0
+  } ]
+}, {
   "id" : "application_1578436911597_0052",
   "name" : "Spark shell",
   "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 d2b3d1b..06015ec 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,19 @@
 [ {
+  "id" : "app-20200706201101-0003",
+  "name" : "Spark shell",
+  "attempts" : [ {
+    "startTime" : "2020-07-07T03:11:00.235GMT",
+    "endTime" : "2020-07-07T03:17:04.231GMT",
+    "lastUpdated" : "",
+    "duration" : 363996,
+    "sparkUser" : "terryk",
+    "completed" : true,
+    "appSparkVersion" : "3.1.0-SNAPSHOT",
+    "endTimeEpoch" : 1594091824231,
+    "startTimeEpoch" : 1594091460235,
+    "lastUpdatedEpoch" : 0
+  } ]
+}, {
   "id" : "application_1578436911597_0052",
   "name" : "Spark shell",
   "attempts" : [ {
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 82489e9..8e6be68 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,19 @@
 [ {
+  "id" : "app-20200706201101-0003",
+  "name" : "Spark shell",
+  "attempts" : [ {
+    "startTime" : "2020-07-07T03:11:00.235GMT",
+    "endTime" : "2020-07-07T03:17:04.231GMT",
+    "lastUpdated" : "",
+    "duration" : 363996,
+    "sparkUser" : "terryk",
+    "completed" : true,
+    "appSparkVersion" : "3.1.0-SNAPSHOT",
+    "lastUpdatedEpoch" : 0,
+    "endTimeEpoch" : 1594091824231,
+    "startTimeEpoch" : 1594091460235
+  } ]
+}, {
   "id" : "application_1578436911597_0052",
   "name" : "Spark shell",
   "attempts" : [ {
@@ -28,19 +43,4 @@
     "lastUpdatedEpoch" : 0,
     "endTimeEpoch" : 1562101355974
   } ]
-}, {
-  "id" : "application_1553914137147_0018",
-  "name" : "LargeBlocks",
-  "attempts" : [ {
-    "startTime" : "2019-04-08T20:39:44.286GMT",
-    "endTime" : "2019-04-08T20:40:46.454GMT",
-    "lastUpdated" : "",
-    "duration" : 62168,
-    "sparkUser" : "systest",
-    "completed" : true,
-    "appSparkVersion" : "3.0.0-SNAPSHOT",
-    "startTimeEpoch" : 1554755984286,
-    "endTimeEpoch" : 1554756046454,
-    "lastUpdatedEpoch" : 0
-  } ]
 } ]
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 ac2bb0e..35d71f9 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,19 @@
 [ {
+  "id" : "app-20200706201101-0003",
+  "name" : "Spark shell",
+  "attempts" : [ {
+    "startTime" : "2020-07-07T03:11:00.235GMT",
+    "endTime" : "2020-07-07T03:17:04.231GMT",
+    "lastUpdated" : "",
+    "duration" : 363996,
+    "sparkUser" : "terryk",
+    "completed" : true,
+    "appSparkVersion" : "3.1.0-SNAPSHOT",
+    "endTimeEpoch" : 1594091824231,
+    "startTimeEpoch" : 1594091460235,
+    "lastUpdatedEpoch" : 0
+  } ]
+}, {
   "id" : "application_1578436911597_0052",
   "name" : "Spark shell",
   "attempts" : [ {
@@ -14,22 +29,20 @@
     "lastUpdatedEpoch" : 0
   } ]
 }, {
-    "id": "application_1555004656427_0144",
-    "name": "Spark shell",
-    "attempts": [
-      {
-        "startTime": "2019-07-02T21:02:17.180GMT",
-        "endTime": "2019-07-02T21:02:35.974GMT",
-        "lastUpdated": "",
-        "duration": 18794,
-        "sparkUser": "tgraves",
-        "completed": true,
-        "appSparkVersion": "3.0.0-SNAPSHOT",
-        "startTimeEpoch": 1562101337180,
-        "lastUpdatedEpoch": 0,
-        "endTimeEpoch": 1562101355974
-      }
-    ]
+  "id" : "application_1555004656427_0144",
+  "name" : "Spark shell",
+  "attempts" : [ {
+    "startTime" : "2019-07-02T21:02:17.180GMT",
+    "endTime" : "2019-07-02T21:02:35.974GMT",
+    "lastUpdated" : "",
+    "duration" : 18794,
+    "sparkUser" : "tgraves",
+    "completed" : true,
+    "appSparkVersion" : "3.0.0-SNAPSHOT",
+    "endTimeEpoch" : 1562101355974,
+    "startTimeEpoch" : 1562101337180,
+    "lastUpdatedEpoch" : 0
+  } ]
 }, {
   "id" : "application_1553914137147_0018",
   "name" : "LargeBlocks",
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 1561676..c6530b1 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,19 @@
 [ {
+  "id" : "app-20200706201101-0003",
+  "name" : "Spark shell",
+  "attempts" : [ {
+    "startTime" : "2020-07-07T03:11:00.235GMT",
+    "endTime" : "2020-07-07T03:17:04.231GMT",
+    "lastUpdated" : "",
+    "duration" : 363996,
+    "sparkUser" : "terryk",
+    "completed" : true,
+    "appSparkVersion" : "3.1.0-SNAPSHOT",
+    "endTimeEpoch" : 1594091824231,
+    "startTimeEpoch" : 1594091460235,
+    "lastUpdatedEpoch" : 0
+  } ]
+}, {
   "id" : "application_1578436911597_0052",
   "name" : "Spark shell",
   "attempts" : [ {
diff --git a/core/src/test/resources/HistoryServerExpectations/stage_list_with_peak_metrics_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_list_with_peak_metrics_expectation.json
new file mode 100644
index 0000000..d455b97
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/stage_list_with_peak_metrics_expectation.json
@@ -0,0 +1,204 @@
+[ {
+  "status" : "COMPLETE",
+  "stageId" : 2,
+  "attemptId" : 0,
+  "numTasks" : 16,
+  "numActiveTasks" : 0,
+  "numCompleteTasks" : 16,
+  "numFailedTasks" : 0,
+  "numKilledTasks" : 0,
+  "numCompletedIndices" : 16,
+  "submissionTime" : "2020-07-07T03:11:21.040GMT",
+  "firstTaskLaunchedTime" : "2020-07-07T03:11:21.077GMT",
+  "completionTime" : "2020-07-07T03:11:23.044GMT",
+  "executorDeserializeTime" : 3905,
+  "executorDeserializeCpuTime" : 979900000,
+  "executorRunTime" : 25579,
+  "executorCpuTime" : 8810338000,
+  "resultSize" : 33883,
+  "jvmGcTime" : 1010,
+  "resultSerializationTime" : 11,
+  "memoryBytesSpilled" : 0,
+  "diskBytesSpilled" : 0,
+  "peakExecutionMemory" : 384640,
+  "inputBytes" : 0,
+  "inputRecords" : 0,
+  "outputBytes" : 0,
+  "outputRecords" : 0,
+  "shuffleRemoteBlocksFetched" : 0,
+  "shuffleLocalBlocksFetched" : 0,
+  "shuffleFetchWaitTime" : 0,
+  "shuffleRemoteBytesRead" : 0,
+  "shuffleRemoteBytesReadToDisk" : 0,
+  "shuffleLocalBytesRead" : 0,
+  "shuffleReadBytes" : 0,
+  "shuffleReadRecords" : 0,
+  "shuffleWriteBytes" : 0,
+  "shuffleWriteTime" : 0,
+  "shuffleWriteRecords" : 0,
+  "name" : "foreach at <console>:26",
+  "details" : "org.apache.spark.sql.Dataset.foreach(Dataset.scala:2862)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:26)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:30)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:32)\n$line19.$read$$iw$$iw$$iw$$iw$$iw.<init>(<console>:34)\n$line19.$read$$iw$$iw$$iw$$iw.<init>(<console>:36)\n$line19.$read$$iw$$iw$$iw.<init>(<console>:38)\n$line19.$read$$iw$$iw.<init>(<console>:40)\n$line19.$read$$iw.<init>(<console>:42)\n$line19.$read.<init>(<console>:44)\n$line19.$read$.<init>(<console>:48)\n$line19.$read$.<clinit>(<console>)\n$line19.$eval$.$print$lzycompute(<console>:7)\n$line19.$eval$.$print(<console>:6)\n$line19.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:745)",
+  "schedulingPool" : "default",
+  "rddIds" : [ 10, 8, 6, 7, 9 ],
+  "accumulatorUpdates" : [ ],
+  "killedTasksSummary" : { },
+  "resourceProfileId" : 0,
+  "peakExecutorMetrics" : {
+    "JVMHeapMemory" : 213367864,
+    "JVMOffHeapMemory" : 189011656,
+    "OnHeapExecutionMemory" : 0,
+    "OffHeapExecutionMemory" : 0,
+    "OnHeapStorageMemory" : 2133349,
+    "OffHeapStorageMemory" : 0,
+    "OnHeapUnifiedMemory" : 2133349,
+    "OffHeapUnifiedMemory" : 0,
+    "DirectPoolMemory" : 282024,
+    "MappedPoolMemory" : 0,
+    "ProcessTreeJVMVMemory" : 0,
+    "ProcessTreeJVMRSSMemory" : 0,
+    "ProcessTreePythonVMemory" : 0,
+    "ProcessTreePythonRSSMemory" : 0,
+    "ProcessTreeOtherVMemory" : 0,
+    "ProcessTreeOtherRSSMemory" : 0,
+    "MinorGCCount" : 13,
+    "MinorGCTime" : 115,
+    "MajorGCCount" : 4,
+    "MajorGCTime" : 339
+  }
+}, {
+  "status" : "COMPLETE",
+  "stageId" : 1,
+  "attemptId" : 0,
+  "numTasks" : 16,
+  "numActiveTasks" : 0,
+  "numCompleteTasks" : 16,
+  "numFailedTasks" : 0,
+  "numKilledTasks" : 0,
+  "numCompletedIndices" : 16,
+  "submissionTime" : "2020-07-07T03:11:20.499GMT",
+  "firstTaskLaunchedTime" : "2020-07-07T03:11:20.502GMT",
+  "completionTime" : "2020-07-07T03:11:20.930GMT",
+  "executorDeserializeTime" : 424,
+  "executorDeserializeCpuTime" : 63666000,
+  "executorRunTime" : 6105,
+  "executorCpuTime" : 426449000,
+  "resultSize" : 31546,
+  "jvmGcTime" : 0,
+  "resultSerializationTime" : 0,
+  "memoryBytesSpilled" : 0,
+  "diskBytesSpilled" : 0,
+  "peakExecutionMemory" : 16256,
+  "inputBytes" : 0,
+  "inputRecords" : 0,
+  "outputBytes" : 0,
+  "outputRecords" : 0,
+  "shuffleRemoteBlocksFetched" : 0,
+  "shuffleLocalBlocksFetched" : 0,
+  "shuffleFetchWaitTime" : 0,
+  "shuffleRemoteBytesRead" : 0,
+  "shuffleRemoteBytesReadToDisk" : 0,
+  "shuffleLocalBytesRead" : 0,
+  "shuffleReadBytes" : 0,
+  "shuffleReadRecords" : 0,
+  "shuffleWriteBytes" : 0,
+  "shuffleWriteTime" : 0,
+  "shuffleWriteRecords" : 0,
+  "name" : "$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266",
+  "description" : "broadcast exchange (runId bac7289a-c1d8-4966-a6a3-d9f347e13a5d)",
+  "details" : "org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:185)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)",
+  "schedulingPool" : "default",
+  "rddIds" : [ 5, 3, 4 ],
+  "accumulatorUpdates" : [ ],
+  "killedTasksSummary" : { },
+  "resourceProfileId" : 0,
+  "peakExecutorMetrics" : {
+    "JVMHeapMemory" : 0,
+    "JVMOffHeapMemory" : 0,
+    "OnHeapExecutionMemory" : 0,
+    "OffHeapExecutionMemory" : 0,
+    "OnHeapStorageMemory" : 0,
+    "OffHeapStorageMemory" : 0,
+    "OnHeapUnifiedMemory" : 0,
+    "OffHeapUnifiedMemory" : 0,
+    "DirectPoolMemory" : 0,
+    "MappedPoolMemory" : 0,
+    "ProcessTreeJVMVMemory" : 0,
+    "ProcessTreeJVMRSSMemory" : 0,
+    "ProcessTreePythonVMemory" : 0,
+    "ProcessTreePythonRSSMemory" : 0,
+    "ProcessTreeOtherVMemory" : 0,
+    "ProcessTreeOtherRSSMemory" : 0,
+    "MinorGCCount" : 0,
+    "MinorGCTime" : 0,
+    "MajorGCCount" : 0,
+    "MajorGCTime" : 0
+  }
+}, {
+  "status" : "COMPLETE",
+  "stageId" : 0,
+  "attemptId" : 0,
+  "numTasks" : 16,
+  "numActiveTasks" : 0,
+  "numCompleteTasks" : 16,
+  "numFailedTasks" : 0,
+  "numKilledTasks" : 0,
+  "numCompletedIndices" : 16,
+  "submissionTime" : "2020-07-07T03:11:18.860GMT",
+  "firstTaskLaunchedTime" : "2020-07-07T03:11:19.253GMT",
+  "completionTime" : "2020-07-07T03:11:20.381GMT",
+  "executorDeserializeTime" : 15399,
+  "executorDeserializeCpuTime" : 3668889000,
+  "executorRunTime" : 1292,
+  "executorCpuTime" : 172863000,
+  "resultSize" : 22375,
+  "jvmGcTime" : 1744,
+  "resultSerializationTime" : 3,
+  "memoryBytesSpilled" : 0,
+  "diskBytesSpilled" : 0,
+  "peakExecutionMemory" : 0,
+  "inputBytes" : 0,
+  "inputRecords" : 0,
+  "outputBytes" : 0,
+  "outputRecords" : 0,
+  "shuffleRemoteBlocksFetched" : 0,
+  "shuffleLocalBlocksFetched" : 0,
+  "shuffleFetchWaitTime" : 0,
+  "shuffleRemoteBytesRead" : 0,
+  "shuffleRemoteBytesReadToDisk" : 0,
+  "shuffleLocalBytesRead" : 0,
+  "shuffleReadBytes" : 0,
+  "shuffleReadRecords" : 0,
+  "shuffleWriteBytes" : 0,
+  "shuffleWriteTime" : 0,
+  "shuffleWriteRecords" : 0,
+  "name" : "$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266",
+  "description" : "broadcast exchange (runId bb0234e5-4157-49a4-b40c-6d538d9f2ec8)",
+  "details" : "org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:185)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)",
+  "schedulingPool" : "default",
+  "rddIds" : [ 2, 0, 1 ],
+  "accumulatorUpdates" : [ ],
+  "killedTasksSummary" : { },
+  "resourceProfileId" : 0,
+  "peakExecutorMetrics" : {
+    "JVMHeapMemory" : 155100856,
+    "JVMOffHeapMemory" : 64239224,
+    "OnHeapExecutionMemory" : 0,
+    "OffHeapExecutionMemory" : 0,
+    "OnHeapStorageMemory" : 6964,
+    "OffHeapStorageMemory" : 0,
+    "OnHeapUnifiedMemory" : 6964,
+    "OffHeapUnifiedMemory" : 0,
+    "DirectPoolMemory" : 1852,
+    "MappedPoolMemory" : 0,
+    "ProcessTreeJVMVMemory" : 0,
+    "ProcessTreeJVMRSSMemory" : 0,
+    "ProcessTreePythonVMemory" : 0,
+    "ProcessTreePythonRSSMemory" : 0,
+    "ProcessTreeOtherVMemory" : 0,
+    "ProcessTreeOtherRSSMemory" : 0,
+    "MinorGCCount" : 7,
+    "MinorGCTime" : 33,
+    "MajorGCCount" : 3,
+    "MajorGCTime" : 110
+  }
+} ]
diff --git a/core/src/test/resources/HistoryServerExpectations/stage_with_peak_metrics_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_with_peak_metrics_expectation.json
new file mode 100644
index 0000000..373510d
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/stage_with_peak_metrics_expectation.json
@@ -0,0 +1,998 @@
+{
+  "status" : "COMPLETE",
+  "stageId" : 2,
+  "attemptId" : 0,
+  "numTasks" : 16,
+  "numActiveTasks" : 0,
+  "numCompleteTasks" : 16,
+  "numFailedTasks" : 0,
+  "numKilledTasks" : 0,
+  "numCompletedIndices" : 16,
+  "submissionTime" : "2020-07-07T03:11:21.040GMT",
+  "firstTaskLaunchedTime" : "2020-07-07T03:11:21.077GMT",
+  "completionTime" : "2020-07-07T03:11:23.044GMT",
+  "executorDeserializeTime" : 3905,
+  "executorDeserializeCpuTime" : 979900000,
+  "executorRunTime" : 25579,
+  "executorCpuTime" : 8810338000,
+  "resultSize" : 33883,
+  "jvmGcTime" : 1010,
+  "resultSerializationTime" : 11,
+  "memoryBytesSpilled" : 0,
+  "diskBytesSpilled" : 0,
+  "peakExecutionMemory" : 384640,
+  "inputBytes" : 0,
+  "inputRecords" : 0,
+  "outputBytes" : 0,
+  "outputRecords" : 0,
+  "shuffleRemoteBlocksFetched" : 0,
+  "shuffleLocalBlocksFetched" : 0,
+  "shuffleFetchWaitTime" : 0,
+  "shuffleRemoteBytesRead" : 0,
+  "shuffleRemoteBytesReadToDisk" : 0,
+  "shuffleLocalBytesRead" : 0,
+  "shuffleReadBytes" : 0,
+  "shuffleReadRecords" : 0,
+  "shuffleWriteBytes" : 0,
+  "shuffleWriteTime" : 0,
+  "shuffleWriteRecords" : 0,
+  "name" : "foreach at <console>:26",
+  "details" : "org.apache.spark.sql.Dataset.foreach(Dataset.scala:2862)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:26)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:30)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:32)\n$line19.$read$$iw$$iw$$iw$$iw$$iw.<init>(<console>:34)\n$line19.$read$$iw$$iw$$iw$$iw.<init>(<console>:36)\n$line19.$read$$iw$$iw$$iw.<init>(<console>:38)\n$line19.$read$$iw$$iw.<init>(<console>:40)\n$line19.$read$$iw.<init>(<console>:42)\n$line19.$read.<init>(<console>:44)\n$line19.$read$.<init>(<console>:48)\n$line19.$read$.<clinit>(<console>)\n$line19.$eval$.$print$lzycompute(<console>:7)\n$line19.$eval$.$print(<console>:6)\n$line19.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:745)",
+  "schedulingPool" : "default",
+  "rddIds" : [ 10, 8, 6, 7, 9 ],
+  "accumulatorUpdates" : [ ],
+  "tasks" : {
+    "42" : {
+      "taskId" : 42,
+      "index" : 10,
+      "attempt" : 0,
+      "launchTime" : "2020-07-07T03:11:21.120GMT",
+      "duration" : 1923,
+      "executorId" : "0",
+      "host" : "127.0.0.1",
+      "status" : "SUCCESS",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 229,
+        "executorDeserializeCpuTime" : 58152000,
+        "executorRunTime" : 1624,
+        "executorCpuTime" : 508230000,
+        "resultSize" : 2115,
+        "jvmGcTime" : 66,
+        "resultSerializationTime" : 0,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0,
+        "peakExecutionMemory" : 24040,
+        "inputMetrics" : {
+          "bytesRead" : 0,
+          "recordsRead" : 0
+        },
+        "outputMetrics" : {
+          "bytesWritten" : 0,
+          "recordsWritten" : 0
+        },
+        "shuffleReadMetrics" : {
+          "remoteBlocksFetched" : 0,
+          "localBlocksFetched" : 0,
+          "fetchWaitTime" : 0,
+          "remoteBytesRead" : 0,
+          "remoteBytesReadToDisk" : 0,
+          "localBytesRead" : 0,
+          "recordsRead" : 0
+        },
+        "shuffleWriteMetrics" : {
+          "bytesWritten" : 0,
+          "writeTime" : 0,
+          "recordsWritten" : 0
+        }
+      },
+      "executorLogs" : {
+        "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout",
+        "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr"
+      },
+      "schedulerDelay" : 70,
+      "gettingResultTime" : 0
+    },
+    "37" : {
+      "taskId" : 37,
+      "index" : 5,
+      "attempt" : 0,
+      "launchTime" : "2020-07-07T03:11:21.100GMT",
+      "duration" : 1915,
+      "executorId" : "0",
+      "host" : "127.0.0.1",
+      "status" : "SUCCESS",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 256,
+        "executorDeserializeCpuTime" : 60890000,
+        "executorRunTime" : 1596,
+        "executorCpuTime" : 507192000,
+        "resultSize" : 2115,
+        "jvmGcTime" : 62,
+        "resultSerializationTime" : 0,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0,
+        "peakExecutionMemory" : 24040,
+        "inputMetrics" : {
+          "bytesRead" : 0,
+          "recordsRead" : 0
+        },
+        "outputMetrics" : {
+          "bytesWritten" : 0,
+          "recordsWritten" : 0
+        },
+        "shuffleReadMetrics" : {
+          "remoteBlocksFetched" : 0,
+          "localBlocksFetched" : 0,
+          "fetchWaitTime" : 0,
+          "remoteBytesRead" : 0,
+          "remoteBytesReadToDisk" : 0,
+          "localBytesRead" : 0,
+          "recordsRead" : 0
+        },
+        "shuffleWriteMetrics" : {
+          "bytesWritten" : 0,
+          "writeTime" : 0,
+          "recordsWritten" : 0
+        }
+      },
+      "executorLogs" : {
+        "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout",
+        "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr"
+      },
+      "schedulerDelay" : 63,
+      "gettingResultTime" : 0
+    },
+    "46" : {
+      "taskId" : 46,
+      "index" : 14,
+      "attempt" : 0,
+      "launchTime" : "2020-07-07T03:11:21.132GMT",
+      "duration" : 1905,
+      "executorId" : "0",
+      "host" : "127.0.0.1",
+      "status" : "SUCCESS",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 218,
+        "executorDeserializeCpuTime" : 51464000,
+        "executorRunTime" : 1618,
+        "executorCpuTime" : 490927000,
+        "resultSize" : 2115,
+        "jvmGcTime" : 66,
+        "resultSerializationTime" : 0,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0,
+        "peakExecutionMemory" : 24040,
+        "inputMetrics" : {
+          "bytesRead" : 0,
+          "recordsRead" : 0
+        },
+        "outputMetrics" : {
+          "bytesWritten" : 0,
+          "recordsWritten" : 0
+        },
+        "shuffleReadMetrics" : {
+          "remoteBlocksFetched" : 0,
+          "localBlocksFetched" : 0,
+          "fetchWaitTime" : 0,
+          "remoteBytesRead" : 0,
+          "remoteBytesReadToDisk" : 0,
+          "localBytesRead" : 0,
+          "recordsRead" : 0
+        },
+        "shuffleWriteMetrics" : {
+          "bytesWritten" : 0,
+          "writeTime" : 0,
+          "recordsWritten" : 0
+        }
+      },
+      "executorLogs" : {
+        "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout",
+        "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr"
+      },
+      "schedulerDelay" : 69,
+      "gettingResultTime" : 0
+    },
+    "38" : {
+      "taskId" : 38,
+      "index" : 6,
+      "attempt" : 0,
+      "launchTime" : "2020-07-07T03:11:21.104GMT",
+      "duration" : 1835,
+      "executorId" : "0",
+      "host" : "127.0.0.1",
+      "status" : "SUCCESS",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 255,
+        "executorDeserializeCpuTime" : 60358000,
+        "executorRunTime" : 1498,
+        "executorCpuTime" : 414110000,
+        "resultSize" : 2158,
+        "jvmGcTime" : 62,
+        "resultSerializationTime" : 11,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0,
+        "peakExecutionMemory" : 24040,
+        "inputMetrics" : {
+          "bytesRead" : 0,
+          "recordsRead" : 0
+        },
+        "outputMetrics" : {
+          "bytesWritten" : 0,
+          "recordsWritten" : 0
+        },
+        "shuffleReadMetrics" : {
+          "remoteBlocksFetched" : 0,
+          "localBlocksFetched" : 0,
+          "fetchWaitTime" : 0,
+          "remoteBytesRead" : 0,
+          "remoteBytesReadToDisk" : 0,
+          "localBytesRead" : 0,
+          "recordsRead" : 0
+        },
+        "shuffleWriteMetrics" : {
+          "bytesWritten" : 0,
+          "writeTime" : 0,
+          "recordsWritten" : 0
+        }
+      },
+      "executorLogs" : {
+        "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout",
+        "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr"
+      },
+      "schedulerDelay" : 71,
+      "gettingResultTime" : 0
+    },
+    "33" : {
+      "taskId" : 33,
+      "index" : 1,
+      "attempt" : 0,
+      "launchTime" : "2020-07-07T03:11:21.082GMT",
+      "duration" : 1943,
+      "executorId" : "0",
+      "host" : "127.0.0.1",
+      "status" : "SUCCESS",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 267,
+        "executorDeserializeCpuTime" : 54442000,
+        "executorRunTime" : 1597,
+        "executorCpuTime" : 519178000,
+        "resultSize" : 2115,
+        "jvmGcTime" : 62,
+        "resultSerializationTime" : 0,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0,
+        "peakExecutionMemory" : 24040,
+        "inputMetrics" : {
+          "bytesRead" : 0,
+          "recordsRead" : 0
+        },
+        "outputMetrics" : {
+          "bytesWritten" : 0,
+          "recordsWritten" : 0
+        },
+        "shuffleReadMetrics" : {
+          "remoteBlocksFetched" : 0,
+          "localBlocksFetched" : 0,
+          "fetchWaitTime" : 0,
+          "remoteBytesRead" : 0,
+          "remoteBytesReadToDisk" : 0,
+          "localBytesRead" : 0,
+          "recordsRead" : 0
+        },
+        "shuffleWriteMetrics" : {
+          "bytesWritten" : 0,
+          "writeTime" : 0,
+          "recordsWritten" : 0
+        }
+      },
+      "executorLogs" : {
+        "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout",
+        "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr"
+      },
+      "schedulerDelay" : 79,
+      "gettingResultTime" : 0
+    },
+    "41" : {
+      "taskId" : 41,
+      "index" : 9,
+      "attempt" : 0,
+      "launchTime" : "2020-07-07T03:11:21.116GMT",
+      "duration" : 1916,
+      "executorId" : "0",
+      "host" : "127.0.0.1",
+      "status" : "SUCCESS",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 240,
+        "executorDeserializeCpuTime" : 55787000,
+        "executorRunTime" : 1614,
+        "executorCpuTime" : 489923000,
+        "resultSize" : 2115,
+        "jvmGcTime" : 66,
+        "resultSerializationTime" : 0,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0,
+        "peakExecutionMemory" : 24040,
+        "inputMetrics" : {
+          "bytesRead" : 0,
+          "recordsRead" : 0
+        },
+        "outputMetrics" : {
+          "bytesWritten" : 0,
+          "recordsWritten" : 0
+        },
+        "shuffleReadMetrics" : {
+          "remoteBlocksFetched" : 0,
+          "localBlocksFetched" : 0,
+          "fetchWaitTime" : 0,
+          "remoteBytesRead" : 0,
+          "remoteBytesReadToDisk" : 0,
+          "localBytesRead" : 0,
+          "recordsRead" : 0
+        },
+        "shuffleWriteMetrics" : {
+          "bytesWritten" : 0,
+          "writeTime" : 0,
+          "recordsWritten" : 0
+        }
+      },
+      "executorLogs" : {
+        "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout",
+        "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr"
+      },
+      "schedulerDelay" : 62,
+      "gettingResultTime" : 0
+    },
+    "32" : {
+      "taskId" : 32,
+      "index" : 0,
+      "attempt" : 0,
+      "launchTime" : "2020-07-07T03:11:21.077GMT",
+      "duration" : 1960,
+      "executorId" : "0",
+      "host" : "127.0.0.1",
+      "status" : "SUCCESS",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 271,
+        "executorDeserializeCpuTime" : 56827000,
+        "executorRunTime" : 1619,
+        "executorCpuTime" : 496683000,
+        "resultSize" : 2115,
+        "jvmGcTime" : 66,
+        "resultSerializationTime" : 0,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0,
+        "peakExecutionMemory" : 24040,
+        "inputMetrics" : {
+          "bytesRead" : 0,
+          "recordsRead" : 0
+        },
+        "outputMetrics" : {
+          "bytesWritten" : 0,
+          "recordsWritten" : 0
+        },
+        "shuffleReadMetrics" : {
+          "remoteBlocksFetched" : 0,
+          "localBlocksFetched" : 0,
+          "fetchWaitTime" : 0,
+          "remoteBytesRead" : 0,
+          "remoteBytesReadToDisk" : 0,
+          "localBytesRead" : 0,
+          "recordsRead" : 0
+        },
+        "shuffleWriteMetrics" : {
+          "bytesWritten" : 0,
+          "writeTime" : 0,
+          "recordsWritten" : 0
+        }
+      },
+      "executorLogs" : {
+        "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout",
+        "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr"
+      },
+      "schedulerDelay" : 70,
+      "gettingResultTime" : 0
+    },
+    "34" : {
+      "taskId" : 34,
+      "index" : 2,
+      "attempt" : 0,
+      "launchTime" : "2020-07-07T03:11:21.087GMT",
+      "duration" : 1939,
+      "executorId" : "0",
+      "host" : "127.0.0.1",
+      "status" : "SUCCESS",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 265,
+        "executorDeserializeCpuTime" : 69492000,
+        "executorRunTime" : 1606,
+        "executorCpuTime" : 508433000,
+        "resultSize" : 2115,
+        "jvmGcTime" : 66,
+        "resultSerializationTime" : 0,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0,
+        "peakExecutionMemory" : 24040,
+        "inputMetrics" : {
+          "bytesRead" : 0,
+          "recordsRead" : 0
+        },
+        "outputMetrics" : {
+          "bytesWritten" : 0,
+          "recordsWritten" : 0
+        },
+        "shuffleReadMetrics" : {
+          "remoteBlocksFetched" : 0,
+          "localBlocksFetched" : 0,
+          "fetchWaitTime" : 0,
+          "remoteBytesRead" : 0,
+          "remoteBytesReadToDisk" : 0,
+          "localBytesRead" : 0,
+          "recordsRead" : 0
+        },
+        "shuffleWriteMetrics" : {
+          "bytesWritten" : 0,
+          "writeTime" : 0,
+          "recordsWritten" : 0
+        }
+      },
+      "executorLogs" : {
+        "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout",
+        "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr"
+      },
+      "schedulerDelay" : 68,
+      "gettingResultTime" : 0
+    },
+    "45" : {
+      "taskId" : 45,
+      "index" : 13,
+      "attempt" : 0,
+      "launchTime" : "2020-07-07T03:11:21.129GMT",
+      "duration" : 1895,
+      "executorId" : "0",
+      "host" : "127.0.0.1",
+      "status" : "SUCCESS",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 221,
+        "executorDeserializeCpuTime" : 54222000,
+        "executorRunTime" : 1595,
+        "executorCpuTime" : 495138000,
+        "resultSize" : 2115,
+        "jvmGcTime" : 62,
+        "resultSerializationTime" : 0,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0,
+        "peakExecutionMemory" : 24040,
+        "inputMetrics" : {
+          "bytesRead" : 0,
+          "recordsRead" : 0
+        },
+        "outputMetrics" : {
+          "bytesWritten" : 0,
+          "recordsWritten" : 0
+        },
+        "shuffleReadMetrics" : {
+          "remoteBlocksFetched" : 0,
+          "localBlocksFetched" : 0,
+          "fetchWaitTime" : 0,
+          "remoteBytesRead" : 0,
+          "remoteBytesReadToDisk" : 0,
+          "localBytesRead" : 0,
+          "recordsRead" : 0
+        },
+        "shuffleWriteMetrics" : {
+          "bytesWritten" : 0,
+          "writeTime" : 0,
+          "recordsWritten" : 0
+        }
+      },
+      "executorLogs" : {
+        "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout",
+        "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr"
+      },
+      "schedulerDelay" : 79,
+      "gettingResultTime" : 0
+    },
+    "44" : {
+      "taskId" : 44,
+      "index" : 12,
+      "attempt" : 0,
+      "launchTime" : "2020-07-07T03:11:21.126GMT",
+      "duration" : 1917,
+      "executorId" : "0",
+      "host" : "127.0.0.1",
+      "status" : "SUCCESS",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 222,
+        "executorDeserializeCpuTime" : 51988000,
+        "executorRunTime" : 1624,
+        "executorCpuTime" : 498187000,
+        "resultSize" : 2115,
+        "jvmGcTime" : 66,
+        "resultSerializationTime" : 0,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0,
+        "peakExecutionMemory" : 24040,
+        "inputMetrics" : {
+          "bytesRead" : 0,
+          "recordsRead" : 0
+        },
+        "outputMetrics" : {
+          "bytesWritten" : 0,
+          "recordsWritten" : 0
+        },
+        "shuffleReadMetrics" : {
+          "remoteBlocksFetched" : 0,
+          "localBlocksFetched" : 0,
+          "fetchWaitTime" : 0,
+          "remoteBytesRead" : 0,
+          "remoteBytesReadToDisk" : 0,
+          "localBytesRead" : 0,
+          "recordsRead" : 0
+        },
+        "shuffleWriteMetrics" : {
+          "bytesWritten" : 0,
+          "writeTime" : 0,
+          "recordsWritten" : 0
+        }
+      },
+      "executorLogs" : {
+        "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout",
+        "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr"
+      },
+      "schedulerDelay" : 71,
+      "gettingResultTime" : 0
+    },
+    "39" : {
+      "taskId" : 39,
+      "index" : 7,
+      "attempt" : 0,
+      "launchTime" : "2020-07-07T03:11:21.109GMT",
+      "duration" : 1915,
+      "executorId" : "0",
+      "host" : "127.0.0.1",
+      "status" : "SUCCESS",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 254,
+        "executorDeserializeCpuTime" : 64380000,
+        "executorRunTime" : 1596,
+        "executorCpuTime" : 539451000,
+        "resultSize" : 2115,
+        "jvmGcTime" : 62,
+        "resultSerializationTime" : 0,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0,
+        "peakExecutionMemory" : 24040,
+        "inputMetrics" : {
+          "bytesRead" : 0,
+          "recordsRead" : 0
+        },
+        "outputMetrics" : {
+          "bytesWritten" : 0,
+          "recordsWritten" : 0
+        },
+        "shuffleReadMetrics" : {
+          "remoteBlocksFetched" : 0,
+          "localBlocksFetched" : 0,
+          "fetchWaitTime" : 0,
+          "remoteBytesRead" : 0,
+          "remoteBytesReadToDisk" : 0,
+          "localBytesRead" : 0,
+          "recordsRead" : 0
+        },
+        "shuffleWriteMetrics" : {
+          "bytesWritten" : 0,
+          "writeTime" : 0,
+          "recordsWritten" : 0
+        }
+      },
+      "executorLogs" : {
+        "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout",
+        "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr"
+      },
+      "schedulerDelay" : 65,
+      "gettingResultTime" : 0
+    },
+    "35" : {
+      "taskId" : 35,
+      "index" : 3,
+      "attempt" : 0,
+      "launchTime" : "2020-07-07T03:11:21.091GMT",
+      "duration" : 1925,
+      "executorId" : "0",
+      "host" : "127.0.0.1",
+      "status" : "SUCCESS",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 263,
+        "executorDeserializeCpuTime" : 62944000,
+        "executorRunTime" : 1598,
+        "executorCpuTime" : 502908000,
+        "resultSize" : 2115,
+        "jvmGcTime" : 62,
+        "resultSerializationTime" : 0,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0,
+        "peakExecutionMemory" : 24040,
+        "inputMetrics" : {
+          "bytesRead" : 0,
+          "recordsRead" : 0
+        },
+        "outputMetrics" : {
+          "bytesWritten" : 0,
+          "recordsWritten" : 0
+        },
+        "shuffleReadMetrics" : {
+          "remoteBlocksFetched" : 0,
+          "localBlocksFetched" : 0,
+          "fetchWaitTime" : 0,
+          "remoteBytesRead" : 0,
+          "remoteBytesReadToDisk" : 0,
+          "localBytesRead" : 0,
+          "recordsRead" : 0
+        },
+        "shuffleWriteMetrics" : {
+          "bytesWritten" : 0,
+          "writeTime" : 0,
+          "recordsWritten" : 0
+        }
+      },
+      "executorLogs" : {
+        "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout",
+        "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr"
+      },
+      "schedulerDelay" : 64,
+      "gettingResultTime" : 0
+    },
+    "43" : {
+      "taskId" : 43,
+      "index" : 11,
+      "attempt" : 0,
+      "launchTime" : "2020-07-07T03:11:21.123GMT",
+      "duration" : 1906,
+      "executorId" : "0",
+      "host" : "127.0.0.1",
+      "status" : "SUCCESS",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 225,
+        "executorDeserializeCpuTime" : 48849000,
+        "executorRunTime" : 1609,
+        "executorCpuTime" : 502120000,
+        "resultSize" : 2115,
+        "jvmGcTime" : 66,
+        "resultSerializationTime" : 0,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0,
+        "peakExecutionMemory" : 24040,
+        "inputMetrics" : {
+          "bytesRead" : 0,
+          "recordsRead" : 0
+        },
+        "outputMetrics" : {
+          "bytesWritten" : 0,
+          "recordsWritten" : 0
+        },
+        "shuffleReadMetrics" : {
+          "remoteBlocksFetched" : 0,
+          "localBlocksFetched" : 0,
+          "fetchWaitTime" : 0,
+          "remoteBytesRead" : 0,
+          "remoteBytesReadToDisk" : 0,
+          "localBytesRead" : 0,
+          "recordsRead" : 0
+        },
+        "shuffleWriteMetrics" : {
+          "bytesWritten" : 0,
+          "writeTime" : 0,
+          "recordsWritten" : 0
+        }
+      },
+      "executorLogs" : {
+        "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout",
+        "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr"
+      },
+      "schedulerDelay" : 72,
+      "gettingResultTime" : 0
+    },
+    "40" : {
+      "taskId" : 40,
+      "index" : 8,
+      "attempt" : 0,
+      "launchTime" : "2020-07-07T03:11:21.112GMT",
+      "duration" : 1904,
+      "executorId" : "0",
+      "host" : "127.0.0.1",
+      "status" : "SUCCESS",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 246,
+        "executorDeserializeCpuTime" : 69760000,
+        "executorRunTime" : 1595,
+        "executorCpuTime" : 510597000,
+        "resultSize" : 2115,
+        "jvmGcTime" : 62,
+        "resultSerializationTime" : 0,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0,
+        "peakExecutionMemory" : 24040,
+        "inputMetrics" : {
+          "bytesRead" : 0,
+          "recordsRead" : 0
+        },
+        "outputMetrics" : {
+          "bytesWritten" : 0,
+          "recordsWritten" : 0
+        },
+        "shuffleReadMetrics" : {
+          "remoteBlocksFetched" : 0,
+          "localBlocksFetched" : 0,
+          "fetchWaitTime" : 0,
+          "remoteBytesRead" : 0,
+          "remoteBytesReadToDisk" : 0,
+          "localBytesRead" : 0,
+          "recordsRead" : 0
+        },
+        "shuffleWriteMetrics" : {
+          "bytesWritten" : 0,
+          "writeTime" : 0,
+          "recordsWritten" : 0
+        }
+      },
+      "executorLogs" : {
+        "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout",
+        "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr"
+      },
+      "schedulerDelay" : 63,
+      "gettingResultTime" : 0
+    },
+    "36" : {
+      "taskId" : 36,
+      "index" : 4,
+      "attempt" : 0,
+      "launchTime" : "2020-07-07T03:11:21.095GMT",
+      "duration" : 1920,
+      "executorId" : "0",
+      "host" : "127.0.0.1",
+      "status" : "SUCCESS",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 260,
+        "executorDeserializeCpuTime" : 112849000,
+        "executorRunTime" : 1596,
+        "executorCpuTime" : 503010000,
+        "resultSize" : 2115,
+        "jvmGcTime" : 62,
+        "resultSerializationTime" : 0,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0,
+        "peakExecutionMemory" : 24040,
+        "inputMetrics" : {
+          "bytesRead" : 0,
+          "recordsRead" : 0
+        },
+        "outputMetrics" : {
+          "bytesWritten" : 0,
+          "recordsWritten" : 0
+        },
+        "shuffleReadMetrics" : {
+          "remoteBlocksFetched" : 0,
+          "localBlocksFetched" : 0,
+          "fetchWaitTime" : 0,
+          "remoteBytesRead" : 0,
+          "remoteBytesReadToDisk" : 0,
+          "localBytesRead" : 0,
+          "recordsRead" : 0
+        },
+        "shuffleWriteMetrics" : {
+          "bytesWritten" : 0,
+          "writeTime" : 0,
+          "recordsWritten" : 0
+        }
+      },
+      "executorLogs" : {
+        "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout",
+        "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr"
+      },
+      "schedulerDelay" : 64,
+      "gettingResultTime" : 0
+    },
+    "47" : {
+      "taskId" : 47,
+      "index" : 15,
+      "attempt" : 0,
+      "launchTime" : "2020-07-07T03:11:21.136GMT",
+      "duration" : 1878,
+      "executorId" : "0",
+      "host" : "127.0.0.1",
+      "status" : "SUCCESS",
+      "taskLocality" : "PROCESS_LOCAL",
+      "speculative" : false,
+      "accumulatorUpdates" : [ ],
+      "taskMetrics" : {
+        "executorDeserializeTime" : 213,
+        "executorDeserializeCpuTime" : 47496000,
+        "executorRunTime" : 1594,
+        "executorCpuTime" : 1324251000,
+        "resultSize" : 2115,
+        "jvmGcTime" : 52,
+        "resultSerializationTime" : 0,
+        "memoryBytesSpilled" : 0,
+        "diskBytesSpilled" : 0,
+        "peakExecutionMemory" : 24040,
+        "inputMetrics" : {
+          "bytesRead" : 0,
+          "recordsRead" : 0
+        },
+        "outputMetrics" : {
+          "bytesWritten" : 0,
+          "recordsWritten" : 0
+        },
+        "shuffleReadMetrics" : {
+          "remoteBlocksFetched" : 0,
+          "localBlocksFetched" : 0,
+          "fetchWaitTime" : 0,
+          "remoteBytesRead" : 0,
+          "remoteBytesReadToDisk" : 0,
+          "localBytesRead" : 0,
+          "recordsRead" : 0
+        },
+        "shuffleWriteMetrics" : {
+          "bytesWritten" : 0,
+          "writeTime" : 0,
+          "recordsWritten" : 0
+        }
+      },
+      "executorLogs" : {
+        "stdout" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout",
+        "stderr" : "http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr"
+      },
+      "schedulerDelay" : 71,
+      "gettingResultTime" : 0
+    }
+  },
+  "executorSummary" : {
+    "0" : {
+      "taskTime" : 30596,
+      "failedTasks" : 0,
+      "succeededTasks" : 16,
+      "killedTasks" : 0,
+      "inputBytes" : 0,
+      "inputRecords" : 0,
+      "outputBytes" : 0,
+      "outputRecords" : 0,
+      "shuffleRead" : 0,
+      "shuffleReadRecords" : 0,
+      "shuffleWrite" : 0,
+      "shuffleWriteRecords" : 0,
+      "memoryBytesSpilled" : 0,
+      "diskBytesSpilled" : 0,
+      "isBlacklistedForStage" : false,
+      "peakMemoryMetrics" : {
+        "JVMHeapMemory" : 0,
+        "JVMOffHeapMemory" : 0,
+        "OnHeapExecutionMemory" : 0,
+        "OffHeapExecutionMemory" : 0,
+        "OnHeapStorageMemory" : 0,
+        "OffHeapStorageMemory" : 0,
+        "OnHeapUnifiedMemory" : 0,
+        "OffHeapUnifiedMemory" : 0,
+        "DirectPoolMemory" : 0,
+        "MappedPoolMemory" : 0,
+        "ProcessTreeJVMVMemory" : 0,
+        "ProcessTreeJVMRSSMemory" : 0,
+        "ProcessTreePythonVMemory" : 0,
+        "ProcessTreePythonRSSMemory" : 0,
+        "ProcessTreeOtherVMemory" : 0,
+        "ProcessTreeOtherRSSMemory" : 0,
+        "MinorGCCount" : 0,
+        "MinorGCTime" : 0,
+        "MajorGCCount" : 0,
+        "MajorGCTime" : 0
+      }
+    },
+    "driver" : {
+      "taskTime" : 0,
+      "failedTasks" : 0,
+      "succeededTasks" : 0,
+      "killedTasks" : 0,
+      "inputBytes" : 0,
+      "inputRecords" : 0,
+      "outputBytes" : 0,
+      "outputRecords" : 0,
+      "shuffleRead" : 0,
+      "shuffleReadRecords" : 0,
+      "shuffleWrite" : 0,
+      "shuffleWriteRecords" : 0,
+      "memoryBytesSpilled" : 0,
+      "diskBytesSpilled" : 0,
+      "isBlacklistedForStage" : false,
+      "peakMemoryMetrics" : {
+        "JVMHeapMemory" : 213367864,
+        "JVMOffHeapMemory" : 189011656,
+        "OnHeapExecutionMemory" : 0,
+        "OffHeapExecutionMemory" : 0,
+        "OnHeapStorageMemory" : 2133349,
+        "OffHeapStorageMemory" : 0,
+        "OnHeapUnifiedMemory" : 2133349,
+        "OffHeapUnifiedMemory" : 0,
+        "DirectPoolMemory" : 282024,
+        "MappedPoolMemory" : 0,
+        "ProcessTreeJVMVMemory" : 0,
+        "ProcessTreeJVMRSSMemory" : 0,
+        "ProcessTreePythonVMemory" : 0,
+        "ProcessTreePythonRSSMemory" : 0,
+        "ProcessTreeOtherVMemory" : 0,
+        "ProcessTreeOtherRSSMemory" : 0,
+        "MinorGCCount" : 13,
+        "MinorGCTime" : 115,
+        "MajorGCCount" : 4,
+        "MajorGCTime" : 339
+      }
+    }
+  },
+  "killedTasksSummary" : { },
+  "resourceProfileId" : 0,
+  "peakExecutorMetrics" : {
+    "JVMHeapMemory" : 213367864,
+    "JVMOffHeapMemory" : 189011656,
+    "OnHeapExecutionMemory" : 0,
+    "OffHeapExecutionMemory" : 0,
+    "OnHeapStorageMemory" : 2133349,
+    "OffHeapStorageMemory" : 0,
+    "OnHeapUnifiedMemory" : 2133349,
+    "OffHeapUnifiedMemory" : 0,
+    "DirectPoolMemory" : 282024,
+    "MappedPoolMemory" : 0,
+    "ProcessTreeJVMVMemory" : 0,
+    "ProcessTreeJVMRSSMemory" : 0,
+    "ProcessTreePythonVMemory" : 0,
+    "ProcessTreePythonRSSMemory" : 0,
+    "ProcessTreeOtherVMemory" : 0,
+    "ProcessTreeOtherRSSMemory" : 0,
+    "MinorGCCount" : 13,
+    "MinorGCTime" : 115,
+    "MajorGCCount" : 4,
+    "MajorGCTime" : 339
+  }
+}
diff --git a/core/src/test/resources/spark-events/app-20200706201101-0003 b/core/src/test/resources/spark-events/app-20200706201101-0003
new file mode 100644
index 0000000..b2923ca
--- /dev/null
+++ b/core/src/test/resources/spark-events/app-20200706201101-0003
@@ -0,0 +1,124 @@
+{"Event":"SparkListenerLogStart","Spark Version":"3.1.0-SNAPSHOT"}
+{"Event":"SparkListenerResourceProfileAdded","Resource Profile Id":0,"Executor Resource Requests":{"cores":{"Resource Name":"cores","Amount":1,"Discovery Script":"","Vendor":""},"memory":{"Resource Name":"memory","Amount":1024,"Discovery Script":"","Vendor":""}},"Task Resource Requests":{"cpus":{"Resource Name":"cpus","Amount":1.0}}}
+{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"localhost","Port":64415},"Maximum Memory":384093388,"Timestamp":1594091461118,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0}
+{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre","Java Version":"1.8.0_231 (Oracle Corporation)","Scala Version":"version 2.12.10"},"Spark Properties":{"spark.driver.host":"localhost","spark.eventLog.enabled":"true","spark.driver.port":"64413","spark.repl.class.uri":"spark://localhost:64413/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/p_/5j1mtw1x0579b34vy4ztcbs40000gn/T/spark-0c713088-21d7-4b35-9b52-7a4dd4d15ae1/repl-361cef81-42cd-4875-a8a6-ce9dfe55682a","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.submit.pyFiles":"","spark.ui.showConsoleProgress":"true","spark.executor.id":"driver","spark.eventLog.logStageExecutorMetrics":"true","spark.submit.deployMode":"client","spark.master":"*********(redacted)","spark.home":"*********(redacted)","spark.sql.catalogImplementation":"in-memory","spark.app.id":"app-20200706201101-0003"},"Hadoop Properties":{"hadoop.service.shutdown.timeout":"30s","yarn.resourcemanager.amlauncher.thread-count":"50","yarn.sharedcache.enabled":"false","fs.s3a.connection.maximum":"15","yarn.nodemanager.numa-awareness.numactl.cmd":"/usr/bin/numactl","fs.s3a.impl":"org.apache.hadoop.fs.s3a.S3AFileSystem","yarn.app.mapreduce.am.scheduler.heartbeat.interval-ms":"1000","yarn.timeline-service.timeline-client.number-of-async-entities-to-merge":"10","hadoop.security.kms.client.timeout":"60","hadoop.http.authentication.kerberos.principal":"HTTP/_HOST@LOCALHOST","mapreduce.jobhistory.loadedjob.tasks.max":"-1","mapreduce.framework.name":"local","yarn.sharedcache.uploader.server.thread-count":"50","yarn.nodemanager.linux-container-executor.nonsecure-mode.user-pattern":"^[_.A-Za-z0-9][-@_.A-Za-z0-9]{0,255}?[$]?$","tfile.fs.output.buffer.size":"262144","yarn.app.mapreduce.am.job.task.listener.thread-count":"30","hadoop.security.groups.cache.background.reload.threads":"3","yarn.resourcemanager.webapp.cross-origin.enabled":"false","fs.AbstractFileSystem.ftp.impl":"org.apache.hadoop.fs.ftp.FtpFs","hadoop.registry.secure":"false","hadoop.shell.safely.delete.limit.num.files":"100","dfs.bytes-per-checksum":"512","mapreduce.job.acl-view-job":" ","fs.s3a.s3guard.ddb.background.sleep":"25ms","fs.s3a.retry.limit":"${fs.s3a.attempts.maximum}","mapreduce.jobhistory.loadedjobs.cache.size":"5","fs.s3a.s3guard.ddb.table.create":"false","yarn.nodemanager.amrmproxy.enabled":"false","yarn.timeline-service.entity-group-fs-store.with-user-dir":"false","mapreduce.input.fileinputformat.split.minsize":"0","yarn.resourcemanager.container.liveness-monitor.interval-ms":"600000","yarn.resourcemanager.client.thread-count":"50","io.seqfile.compress.blocksize":"1000000","yarn.sharedcache.checksum.algo.impl":"org.apache.hadoop.yarn.sharedcache.ChecksumSHA256Impl","yarn.nodemanager.amrmproxy.interceptor-class.pipeline":"org.apache.hadoop.yarn.server.nodemanager.amrmproxy.DefaultRequestInterceptor","yarn.timeline-service.entity-group-fs-store.leveldb-cache-read-cache-size":"10485760","mapreduce.reduce.shuffle.fetch.retry.interval-ms":"1000","mapreduce.task.profile.maps":"0-2","yarn.scheduler.include-port-in-node-name":"false","yarn.nodemanager.admin-env":"MALLOC_ARENA_MAX=$MALLOC_ARENA_MAX","yarn.resourcemanager.node-removal-untracked.timeout-ms":"60000","mapreduce.am.max-attempts":"2","hadoop.security.kms.client.failover.sleep.base.millis":"100","mapreduce.jobhistory.webapp.https.address":"0.0.0.0:19890","yarn.node-labels.fs-store.impl.class":"org.apache.hadoop.yarn.nodelabels.FileSystemNodeLabelsStore","yarn.nodemanager.collector-service.address":"${yarn.nodemanager.hostname}:8048","fs.trash.checkpoint.interval":"0","mapreduce.job.map.output.collector.class":"org.apache.hadoop.mapred.MapTask$MapOutputBuffer","yarn.resourcemanager.node-ip-cache.expiry-interval-secs":"-1","hadoop.http.authentication.signature.secret.file":"*********(redacted)","hadoop.jetty.logs.serve.aliases":"true","yarn.resourcemanager.placement-constraints.handler":"disabled","yarn.timeline-service.handler-thread-count":"10","yarn.resourcemanager.max-completed-applications":"1000","yarn.resourcemanager.system-metrics-publisher.enabled":"false","yarn.resourcemanager.placement-constraints.algorithm.class":"org.apache.hadoop.yarn.server.resourcemanager.scheduler.constraint.algorithm.DefaultPlacementAlgorithm","yarn.sharedcache.webapp.address":"0.0.0.0:8788","yarn.resourcemanager.delegation.token.renew-interval":"*********(redacted)","yarn.sharedcache.nm.uploader.replication.factor":"10","hadoop.security.groups.negative-cache.secs":"30","yarn.app.mapreduce.task.container.log.backups":"0","mapreduce.reduce.skip.proc-count.auto-incr":"true","hadoop.security.group.mapping.ldap.posix.attr.gid.name":"gidNumber","ipc.client.fallback-to-simple-auth-allowed":"false","yarn.nodemanager.resource.memory.enforced":"true","yarn.client.failover-proxy-provider":"org.apache.hadoop.yarn.client.ConfiguredRMFailoverProxyProvider","yarn.timeline-service.http-authentication.simple.anonymous.allowed":"true","ha.health-monitor.check-interval.ms":"1000","yarn.acl.reservation-enable":"false","yarn.resourcemanager.store.class":"org.apache.hadoop.yarn.server.resourcemanager.recovery.FileSystemRMStateStore","yarn.app.mapreduce.am.hard-kill-timeout-ms":"10000","fs.s3a.etag.checksum.enabled":"false","yarn.nodemanager.container-metrics.enable":"true","yarn.timeline-service.client.fd-clean-interval-secs":"60","yarn.resourcemanager.nodemanagers.heartbeat-interval-ms":"1000","hadoop.common.configuration.version":"3.0.0","fs.s3a.s3guard.ddb.table.capacity.read":"500","yarn.nodemanager.remote-app-log-dir-suffix":"logs","yarn.nodemanager.windows-container.cpu-limit.enabled":"false","yarn.nodemanager.runtime.linux.docker.privileged-containers.allowed":"false","file.blocksize":"67108864","hadoop.registry.zk.retry.ceiling.ms":"60000","yarn.scheduler.configuration.leveldb-store.path":"${hadoop.tmp.dir}/yarn/system/confstore","yarn.sharedcache.store.in-memory.initial-delay-mins":"10","mapreduce.jobhistory.principal":"jhs/_HOST@REALM.TLD","mapreduce.map.skip.proc-count.auto-incr":"true","fs.s3a.committer.name":"file","mapreduce.task.profile.reduces":"0-2","hadoop.zk.num-retries":"1000","yarn.webapp.xfs-filter.enabled":"true","seq.io.sort.mb":"100","yarn.scheduler.configuration.max.version":"100","yarn.timeline-service.webapp.https.address":"${yarn.timeline-service.hostname}:8190","yarn.resourcemanager.scheduler.address":"${yarn.resourcemanager.hostname}:8030","yarn.node-labels.enabled":"false","yarn.resourcemanager.webapp.ui-actions.enabled":"true","mapreduce.task.timeout":"600000","yarn.sharedcache.client-server.thread-count":"50","hadoop.security.groups.shell.command.timeout":"0s","hadoop.security.crypto.cipher.suite":"AES/CTR/NoPadding","yarn.nodemanager.elastic-memory-control.oom-handler":"org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.DefaultOOMHandler","yarn.resourcemanager.connect.max-wait.ms":"900000","fs.defaultFS":"file:///","yarn.minicluster.use-rpc":"false","fs.har.impl.disable.cache":"true","yarn.webapp.ui2.enable":"false","io.compression.codec.bzip2.library":"system-native","yarn.nodemanager.distributed-scheduling.enabled":"false","mapreduce.shuffle.connection-keep-alive.timeout":"5","yarn.resourcemanager.webapp.https.address":"${yarn.resourcemanager.hostname}:8090","mapreduce.jobhistory.address":"0.0.0.0:10020","yarn.resourcemanager.nm-tokens.master-key-rolling-interval-secs":"*********(redacted)","yarn.is.minicluster":"false","yarn.nodemanager.address":"${yarn.nodemanager.hostname}:0","fs.abfss.impl":"org.apache.hadoop.fs.azurebfs.SecureAzureBlobFileSystem","fs.AbstractFileSystem.s3a.impl":"org.apache.hadoop.fs.s3a.S3A","mapreduce.task.combine.progress.records":"10000","yarn.resourcemanager.epoch.range":"0","yarn.resourcemanager.am.max-attempts":"2","yarn.nodemanager.linux-container-executor.cgroups.hierarchy":"/hadoop-yarn","fs.AbstractFileSystem.wasbs.impl":"org.apache.hadoop.fs.azure.Wasbs","yarn.timeline-service.entity-group-fs-store.cache-store-class":"org.apache.hadoop.yarn.server.timeline.MemoryTimelineStore","fs.ftp.transfer.mode":"BLOCK_TRANSFER_MODE","ipc.server.log.slow.rpc":"false","yarn.resourcemanager.node-labels.provider.fetch-interval-ms":"1800000","yarn.router.webapp.https.address":"0.0.0.0:8091","yarn.nodemanager.webapp.cross-origin.enabled":"false","fs.wasb.impl":"org.apache.hadoop.fs.azure.NativeAzureFileSystem","yarn.resourcemanager.auto-update.containers":"false","yarn.app.mapreduce.am.job.committer.cancel-timeout":"60000","yarn.scheduler.configuration.zk-store.parent-path":"/confstore","yarn.nodemanager.default-container-executor.log-dirs.permissions":"710","yarn.app.attempt.diagnostics.limit.kc":"64","ftp.bytes-per-checksum":"512","yarn.nodemanager.resource.memory-mb":"-1","fs.AbstractFileSystem.abfs.impl":"org.apache.hadoop.fs.azurebfs.Abfs","yarn.timeline-service.writer.flush-interval-seconds":"60","fs.s3a.fast.upload.active.blocks":"4","hadoop.security.credential.clear-text-fallback":"true","yarn.nodemanager.collector-service.thread-count":"5","fs.azure.secure.mode":"false","mapreduce.jobhistory.joblist.cache.size":"20000","fs.ftp.host":"0.0.0.0","yarn.resourcemanager.fs.state-store.num-retries":"0","yarn.resourcemanager.nodemanager-connect-retries":"10","yarn.nodemanager.log-aggregation.num-log-files-per-app":"30","hadoop.security.kms.client.encrypted.key.cache.low-watermark":"0.3f","fs.s3a.committer.magic.enabled":"false","yarn.timeline-service.client.max-retries":"30","dfs.ha.fencing.ssh.connect-timeout":"30000","yarn.log-aggregation-enable":"false","yarn.system-metrics-publisher.enabled":"false","mapreduce.reduce.markreset.buffer.percent":"0.0","fs.AbstractFileSystem.viewfs.impl":"org.apache.hadoop.fs.viewfs.ViewFs","mapreduce.task.io.sort.factor":"10","yarn.nodemanager.amrmproxy.client.thread-count":"25","ha.failover-controller.new-active.rpc-timeout.ms":"60000","yarn.nodemanager.container-localizer.java.opts":"-Xmx256m","mapreduce.jobhistory.datestring.cache.size":"200000","mapreduce.job.acl-modify-job":" ","yarn.nodemanager.windows-container.memory-limit.enabled":"false","yarn.timeline-service.webapp.address":"${yarn.timeline-service.hostname}:8188","yarn.app.mapreduce.am.job.committer.commit-window":"10000","yarn.nodemanager.container-manager.thread-count":"20","yarn.minicluster.fixed.ports":"false","hadoop.tags.system":"YARN,HDFS,NAMENODE,DATANODE,REQUIRED,SECURITY,KERBEROS,PERFORMANCE,CLIENT\n      ,SERVER,DEBUG,DEPRECATED,COMMON,OPTIONAL","yarn.cluster.max-application-priority":"0","yarn.timeline-service.ttl-enable":"true","mapreduce.jobhistory.recovery.store.fs.uri":"${hadoop.tmp.dir}/mapred/history/recoverystore","hadoop.caller.context.signature.max.size":"40","yarn.client.load.resource-types.from-server":"false","ha.zookeeper.session-timeout.ms":"10000","tfile.io.chunk.size":"1048576","fs.s3a.s3guard.ddb.table.capacity.write":"100","mapreduce.job.speculative.slowtaskthreshold":"1.0","io.serializations":"org.apache.hadoop.io.serializer.WritableSerialization, org.apache.hadoop.io.serializer.avro.AvroSpecificSerialization, org.apache.hadoop.io.serializer.avro.AvroReflectSerialization","hadoop.security.kms.client.failover.sleep.max.millis":"2000","hadoop.security.group.mapping.ldap.directory.search.timeout":"10000","yarn.scheduler.configuration.store.max-logs":"1000","yarn.nodemanager.node-attributes.provider.fetch-interval-ms":"600000","fs.swift.impl":"org.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem","yarn.nodemanager.local-cache.max-files-per-directory":"8192","hadoop.http.cross-origin.enabled":"false","hadoop.zk.acl":"world:anyone:rwcda","mapreduce.map.sort.spill.percent":"0.80","yarn.timeline-service.entity-group-fs-store.scan-interval-seconds":"60","yarn.node-attribute.fs-store.impl.class":"org.apache.hadoop.yarn.server.resourcemanager.nodelabels.FileSystemNodeAttributeStore","fs.s3a.retry.interval":"500ms","yarn.timeline-service.client.best-effort":"false","yarn.resourcemanager.webapp.delegation-token-auth-filter.enabled":"*********(redacted)","hadoop.security.group.mapping.ldap.posix.attr.uid.name":"uidNumber","fs.AbstractFileSystem.swebhdfs.impl":"org.apache.hadoop.fs.SWebHdfs","yarn.nodemanager.elastic-memory-control.timeout-sec":"5","mapreduce.ifile.readahead":"true","yarn.timeline-service.leveldb-timeline-store.ttl-interval-ms":"300000","yarn.timeline-service.reader.webapp.address":"${yarn.timeline-service.webapp.address}","yarn.resourcemanager.placement-constraints.algorithm.pool-size":"1","yarn.timeline-service.hbase.coprocessor.jar.hdfs.location":"/hbase/coprocessor/hadoop-yarn-server-timelineservice.jar","hadoop.security.kms.client.encrypted.key.cache.num.refill.threads":"2","yarn.resourcemanager.scheduler.class":"org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler","yarn.app.mapreduce.am.command-opts":"-Xmx1024m","mapreduce.cluster.local.dir":"${hadoop.tmp.dir}/mapred/local","io.mapfile.bloom.error.rate":"0.005","fs.client.resolve.topology.enabled":"false","yarn.nodemanager.runtime.linux.allowed-runtimes":"default","yarn.sharedcache.store.class":"org.apache.hadoop.yarn.server.sharedcachemanager.store.InMemorySCMStore","ha.failover-controller.graceful-fence.rpc-timeout.ms":"5000","ftp.replication":"3","hadoop.security.uid.cache.secs":"14400","mapreduce.job.maxtaskfailures.per.tracker":"3","fs.s3a.metadatastore.impl":"org.apache.hadoop.fs.s3a.s3guard.NullMetadataStore","io.skip.checksum.errors":"false","yarn.app.mapreduce.client-am.ipc.max-retries-on-timeouts":"3","yarn.timeline-service.webapp.xfs-filter.xframe-options":"SAMEORIGIN","fs.s3a.connection.timeout":"200000","mapreduce.job.max.split.locations":"15","yarn.resourcemanager.nm-container-queuing.max-queue-length":"15","hadoop.registry.zk.session.timeout.ms":"60000","yarn.federation.cache-ttl.secs":"300","mapreduce.jvm.system-properties-to-log":"os.name,os.version,java.home,java.runtime.version,java.vendor,java.version,java.vm.name,java.class.path,java.io.tmpdir,user.dir,user.name","yarn.resourcemanager.opportunistic-container-allocation.nodes-used":"10","yarn.timeline-service.entity-group-fs-store.active-dir":"/tmp/entity-file-history/active","mapreduce.shuffle.transfer.buffer.size":"131072","yarn.timeline-service.client.retry-interval-ms":"1000","yarn.http.policy":"HTTP_ONLY","fs.s3a.socket.send.buffer":"8192","fs.AbstractFileSystem.abfss.impl":"org.apache.hadoop.fs.azurebfs.Abfss","yarn.sharedcache.uploader.server.address":"0.0.0.0:8046","yarn.resourcemanager.delegation-token.max-conf-size-bytes":"*********(redacted)","hadoop.http.authentication.token.validity":"*********(redacted)","mapreduce.shuffle.max.connections":"0","yarn.minicluster.yarn.nodemanager.resource.memory-mb":"4096","mapreduce.job.emit-timeline-data":"false","yarn.nodemanager.resource.system-reserved-memory-mb":"-1","hadoop.kerberos.min.seconds.before.relogin":"60","mapreduce.jobhistory.move.thread-count":"3","yarn.resourcemanager.admin.client.thread-count":"1","yarn.dispatcher.drain-events.timeout":"300000","fs.s3a.buffer.dir":"${hadoop.tmp.dir}/s3a","hadoop.ssl.enabled.protocols":"TLSv1,SSLv2Hello,TLSv1.1,TLSv1.2","mapreduce.jobhistory.admin.address":"0.0.0.0:10033","yarn.log-aggregation-status.time-out.ms":"600000","fs.s3a.assumed.role.sts.endpoint.region":"us-west-1","mapreduce.shuffle.port":"13562","yarn.resourcemanager.max-log-aggregation-diagnostics-in-memory":"10","yarn.nodemanager.health-checker.interval-ms":"600000","yarn.router.clientrm.interceptor-class.pipeline":"org.apache.hadoop.yarn.server.router.clientrm.DefaultClientRequestInterceptor","yarn.resourcemanager.zk-appid-node.split-index":"0","ftp.blocksize":"67108864","yarn.nodemanager.runtime.linux.sandbox-mode.local-dirs.permissions":"read","yarn.router.rmadmin.interceptor-class.pipeline":"org.apache.hadoop.yarn.server.router.rmadmin.DefaultRMAdminRequestInterceptor","yarn.nodemanager.log-container-debug-info.enabled":"true","yarn.client.max-cached-nodemanagers-proxies":"0","yarn.nodemanager.linux-container-executor.cgroups.delete-delay-ms":"20","yarn.nodemanager.delete.debug-delay-sec":"0","yarn.nodemanager.pmem-check-enabled":"true","yarn.nodemanager.disk-health-checker.max-disk-utilization-per-disk-percentage":"90.0","mapreduce.app-submission.cross-platform":"false","yarn.resourcemanager.work-preserving-recovery.scheduling-wait-ms":"10000","yarn.nodemanager.container-retry-minimum-interval-ms":"1000","hadoop.security.groups.cache.secs":"300","yarn.federation.enabled":"false","fs.azure.local.sas.key.mode":"false","ipc.maximum.data.length":"67108864","mapreduce.shuffle.max.threads":"0","yarn.router.pipeline.cache-max-size":"25","yarn.resourcemanager.nm-container-queuing.load-comparator":"QUEUE_LENGTH","hadoop.security.authorization":"false","mapreduce.job.complete.cancel.delegation.tokens":"*********(redacted)","fs.s3a.paging.maximum":"5000","nfs.exports.allowed.hosts":"* rw","yarn.nodemanager.amrmproxy.ha.enable":"false","mapreduce.jobhistory.http.policy":"HTTP_ONLY","yarn.sharedcache.store.in-memory.check-period-mins":"720","hadoop.security.group.mapping.ldap.ssl":"false","yarn.client.application-client-protocol.poll-interval-ms":"200","yarn.scheduler.configuration.leveldb-store.compaction-interval-secs":"86400","yarn.timeline-service.writer.class":"org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineWriterImpl","ha.zookeeper.parent-znode":"/hadoop-ha","yarn.nodemanager.log-aggregation.policy.class":"org.apache.hadoop.yarn.server.nodemanager.containermanager.logaggregation.AllContainerLogAggregationPolicy","mapreduce.reduce.shuffle.merge.percent":"0.66","hadoop.security.group.mapping.ldap.search.filter.group":"(objectClass=group)","yarn.resourcemanager.placement-constraints.scheduler.pool-size":"1","yarn.nodemanager.resourcemanager.minimum.version":"NONE","mapreduce.job.speculative.speculative-cap-running-tasks":"0.1","yarn.admin.acl":"*","yarn.nodemanager.recovery.supervised":"false","yarn.sharedcache.admin.thread-count":"1","yarn.resourcemanager.ha.automatic-failover.enabled":"true","mapreduce.reduce.skip.maxgroups":"0","mapreduce.reduce.shuffle.connect.timeout":"180000","yarn.resourcemanager.address":"${yarn.resourcemanager.hostname}:8032","ipc.client.ping":"true","mapreduce.task.local-fs.write-limit.bytes":"-1","fs.adl.oauth2.access.token.provider.type":"*********(redacted)","mapreduce.shuffle.ssl.file.buffer.size":"65536","yarn.resourcemanager.ha.automatic-failover.embedded":"true","yarn.nodemanager.resource-plugins.gpu.docker-plugin":"nvidia-docker-v1","hadoop.ssl.enabled":"false","fs.s3a.multipart.purge":"false","yarn.scheduler.configuration.store.class":"file","yarn.resourcemanager.nm-container-queuing.queue-limit-stdev":"1.0f","mapreduce.job.end-notification.max.attempts":"5","mapreduce.output.fileoutputformat.compress.codec":"org.apache.hadoop.io.compress.DefaultCodec","yarn.nodemanager.container-monitor.procfs-tree.smaps-based-rss.enabled":"false","ipc.client.bind.wildcard.addr":"false","yarn.resourcemanager.webapp.rest-csrf.enabled":"false","ha.health-monitor.connect-retry-interval.ms":"1000","yarn.nodemanager.keytab":"/etc/krb5.keytab","mapreduce.jobhistory.keytab":"/etc/security/keytab/jhs.service.keytab","fs.s3a.threads.max":"10","mapreduce.reduce.shuffle.input.buffer.percent":"0.70","yarn.nodemanager.runtime.linux.docker.allowed-container-networks":"host,none,bridge","yarn.nodemanager.node-labels.resync-interval-ms":"120000","hadoop.tmp.dir":"/tmp/hadoop-${user.name}","mapreduce.job.maps":"2","mapreduce.jobhistory.webapp.rest-csrf.custom-header":"X-XSRF-Header","mapreduce.job.end-notification.max.retry.interval":"5000","yarn.log-aggregation.retain-check-interval-seconds":"-1","yarn.resourcemanager.resource-tracker.client.thread-count":"50","yarn.rm.system-metrics-publisher.emit-container-events":"false","yarn.timeline-service.leveldb-timeline-store.start-time-read-cache-size":"10000","yarn.resourcemanager.ha.automatic-failover.zk-base-path":"/yarn-leader-election","io.seqfile.local.dir":"${hadoop.tmp.dir}/io/local","fs.s3a.s3guard.ddb.throttle.retry.interval":"100ms","fs.AbstractFileSystem.wasb.impl":"org.apache.hadoop.fs.azure.Wasb","mapreduce.client.submit.file.replication":"10","mapreduce.jobhistory.minicluster.fixed.ports":"false","fs.s3a.multipart.threshold":"2147483647","yarn.resourcemanager.webapp.xfs-filter.xframe-options":"SAMEORIGIN","mapreduce.jobhistory.done-dir":"${yarn.app.mapreduce.am.staging-dir}/history/done","ipc.client.idlethreshold":"4000","yarn.nodemanager.linux-container-executor.cgroups.strict-resource-usage":"false","mapreduce.reduce.input.buffer.percent":"0.0","yarn.nodemanager.runtime.linux.docker.userremapping-gid-threshold":"1","yarn.nodemanager.webapp.rest-csrf.enabled":"false","fs.ftp.host.port":"21","ipc.ping.interval":"60000","yarn.resourcemanager.history-writer.multi-threaded-dispatcher.pool-size":"10","yarn.resourcemanager.admin.address":"${yarn.resourcemanager.hostname}:8033","file.client-write-packet-size":"65536","ipc.client.kill.max":"10","mapreduce.reduce.speculative":"true","hadoop.security.key.default.bitlength":"128","mapreduce.job.reducer.unconditional-preempt.delay.sec":"300","yarn.nodemanager.disk-health-checker.interval-ms":"120000","yarn.nodemanager.log.deletion-threads-count":"4","yarn.webapp.filter-entity-list-by-user":"false","ipc.client.connection.maxidletime":"10000","mapreduce.task.io.sort.mb":"100","yarn.nodemanager.localizer.client.thread-count":"5","io.erasurecode.codec.rs.rawcoders":"rs_native,rs_java","io.erasurecode.codec.rs-legacy.rawcoders":"rs-legacy_java","yarn.sharedcache.admin.address":"0.0.0.0:8047","yarn.resourcemanager.placement-constraints.algorithm.iterator":"SERIAL","yarn.nodemanager.localizer.cache.cleanup.interval-ms":"600000","hadoop.security.crypto.codec.classes.aes.ctr.nopadding":"org.apache.hadoop.crypto.OpensslAesCtrCryptoCodec, org.apache.hadoop.crypto.JceAesCtrCryptoCodec","mapreduce.job.cache.limit.max-resources-mb":"0","fs.s3a.connection.ssl.enabled":"true","yarn.nodemanager.process-kill-wait.ms":"5000","mapreduce.job.hdfs-servers":"${fs.defaultFS}","hadoop.workaround.non.threadsafe.getpwuid":"true","fs.df.interval":"60000","fs.s3a.multiobjectdelete.enable":"true","yarn.sharedcache.cleaner.resource-sleep-ms":"0","yarn.nodemanager.disk-health-checker.min-healthy-disks":"0.25","hadoop.shell.missing.defaultFs.warning":"false","io.file.buffer.size":"65536","hadoop.security.group.mapping.ldap.search.attr.member":"member","hadoop.security.random.device.file.path":"/dev/urandom","hadoop.security.sensitive-config-keys":"*********(redacted)","fs.s3a.s3guard.ddb.max.retries":"9","hadoop.rpc.socket.factory.class.default":"org.apache.hadoop.net.StandardSocketFactory","yarn.intermediate-data-encryption.enable":"false","yarn.resourcemanager.connect.retry-interval.ms":"30000","yarn.nodemanager.container.stderr.pattern":"{*stderr*,*STDERR*}","yarn.scheduler.minimum-allocation-mb":"1024","yarn.app.mapreduce.am.staging-dir":"/tmp/hadoop-yarn/staging","mapreduce.reduce.shuffle.read.timeout":"180000","hadoop.http.cross-origin.max-age":"1800","io.erasurecode.codec.xor.rawcoders":"xor_native,xor_java","fs.s3a.connection.establish.timeout":"5000","mapreduce.job.running.map.limit":"0","yarn.minicluster.control-resource-monitoring":"false","hadoop.ssl.require.client.cert":"false","hadoop.kerberos.kinit.command":"kinit","yarn.federation.state-store.class":"org.apache.hadoop.yarn.server.federation.store.impl.MemoryFederationStateStore","mapreduce.reduce.log.level":"INFO","hadoop.security.dns.log-slow-lookups.threshold.ms":"1000","mapreduce.job.ubertask.enable":"false","adl.http.timeout":"-1","yarn.resourcemanager.placement-constraints.retry-attempts":"3","hadoop.caller.context.enabled":"false","yarn.nodemanager.vmem-pmem-ratio":"2.1","hadoop.rpc.protection":"authentication","ha.health-monitor.rpc-timeout.ms":"45000","yarn.nodemanager.remote-app-log-dir":"/tmp/logs","hadoop.zk.timeout-ms":"10000","fs.s3a.s3guard.cli.prune.age":"86400000","yarn.nodemanager.resource.pcores-vcores-multiplier":"1.0","yarn.nodemanager.runtime.linux.sandbox-mode":"disabled","yarn.app.mapreduce.am.containerlauncher.threadpool-initial-size":"10","fs.s3a.committer.threads":"8","hadoop.zk.retry-interval-ms":"1000","hadoop.security.crypto.buffer.size":"8192","yarn.nodemanager.node-labels.provider.fetch-interval-ms":"600000","mapreduce.jobhistory.recovery.store.leveldb.path":"${hadoop.tmp.dir}/mapred/history/recoverystore","yarn.client.failover-retries-on-socket-timeouts":"0","yarn.nodemanager.resource.memory.enabled":"false","fs.azure.authorization.caching.enable":"true","hadoop.security.instrumentation.requires.admin":"false","yarn.nodemanager.delete.thread-count":"4","mapreduce.job.finish-when-all-reducers-done":"true","hadoop.registry.jaas.context":"Client","yarn.timeline-service.leveldb-timeline-store.path":"${hadoop.tmp.dir}/yarn/timeline","io.map.index.interval":"128","yarn.resourcemanager.nm-container-queuing.max-queue-wait-time-ms":"100","fs.abfs.impl":"org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem","mapreduce.job.counters.max":"120","mapreduce.jobhistory.webapp.rest-csrf.enabled":"false","yarn.timeline-service.store-class":"org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore","mapreduce.jobhistory.move.interval-ms":"180000","yarn.nodemanager.localizer.fetch.thread-count":"4","yarn.resourcemanager.scheduler.client.thread-count":"50","hadoop.ssl.hostname.verifier":"DEFAULT","yarn.timeline-service.leveldb-state-store.path":"${hadoop.tmp.dir}/yarn/timeline","mapreduce.job.classloader":"false","mapreduce.task.profile.map.params":"${mapreduce.task.profile.params}","ipc.client.connect.timeout":"20000","hadoop.security.auth_to_local.mechanism":"hadoop","yarn.timeline-service.app-collector.linger-period.ms":"60000","yarn.nm.liveness-monitor.expiry-interval-ms":"600000","yarn.resourcemanager.reservation-system.planfollower.time-step":"1000","yarn.nodemanager.runtime.linux.docker.enable-userremapping.allowed":"true","yarn.webapp.api-service.enable":"false","yarn.nodemanager.recovery.enabled":"false","mapreduce.job.end-notification.retry.interval":"1000","fs.du.interval":"600000","fs.ftp.impl":"org.apache.hadoop.fs.ftp.FTPFileSystem","yarn.nodemanager.container.stderr.tail.bytes":"4096","hadoop.security.group.mapping.ldap.read.timeout.ms":"60000","hadoop.security.groups.cache.warn.after.ms":"5000","file.bytes-per-checksum":"512","mapreduce.outputcommitter.factory.scheme.s3a":"org.apache.hadoop.fs.s3a.commit.S3ACommitterFactory","hadoop.security.groups.cache.background.reload":"false","yarn.nodemanager.container-monitor.enabled":"true","yarn.nodemanager.elastic-memory-control.enabled":"false","net.topology.script.number.args":"100","mapreduce.task.merge.progress.records":"10000","yarn.nodemanager.localizer.address":"${yarn.nodemanager.hostname}:8040","yarn.timeline-service.keytab":"/etc/krb5.keytab","mapreduce.reduce.shuffle.fetch.retry.timeout-ms":"30000","yarn.resourcemanager.rm.container-allocation.expiry-interval-ms":"600000","mapreduce.fileoutputcommitter.algorithm.version":"2","yarn.resourcemanager.work-preserving-recovery.enabled":"true","mapreduce.map.skip.maxrecords":"0","yarn.sharedcache.root-dir":"/sharedcache","fs.s3a.retry.throttle.limit":"${fs.s3a.attempts.maximum}","hadoop.http.authentication.type":"simple","mapreduce.job.cache.limit.max-resources":"0","mapreduce.task.userlog.limit.kb":"0","yarn.resourcemanager.scheduler.monitor.enable":"false","ipc.client.connect.max.retries":"10","hadoop.registry.zk.retry.times":"5","yarn.nodemanager.resource-monitor.interval-ms":"3000","yarn.nodemanager.resource-plugins.gpu.allowed-gpu-devices":"auto","mapreduce.job.sharedcache.mode":"disabled","yarn.nodemanager.webapp.rest-csrf.custom-header":"X-XSRF-Header","mapreduce.shuffle.listen.queue.size":"128","yarn.scheduler.configuration.mutation.acl-policy.class":"org.apache.hadoop.yarn.server.resourcemanager.scheduler.DefaultConfigurationMutationACLPolicy","mapreduce.map.cpu.vcores":"1","yarn.log-aggregation.file-formats":"TFile","yarn.timeline-service.client.fd-retain-secs":"300","hadoop.user.group.static.mapping.overrides":"dr.who=;","fs.azure.sas.expiry.period":"90d","mapreduce.jobhistory.recovery.store.class":"org.apache.hadoop.mapreduce.v2.hs.HistoryServerFileSystemStateStoreService","yarn.resourcemanager.fail-fast":"${yarn.fail-fast}","yarn.resourcemanager.proxy-user-privileges.enabled":"false","yarn.router.webapp.interceptor-class.pipeline":"org.apache.hadoop.yarn.server.router.webapp.DefaultRequestInterceptorREST","yarn.nodemanager.resource.memory.cgroups.soft-limit-percentage":"90.0","mapreduce.job.reducer.preempt.delay.sec":"0","hadoop.util.hash.type":"murmur","yarn.nodemanager.disk-validator":"basic","yarn.app.mapreduce.client.job.max-retries":"3","mapreduce.reduce.shuffle.retry-delay.max.ms":"60000","hadoop.security.group.mapping.ldap.connection.timeout.ms":"60000","mapreduce.task.profile.params":"-agentlib:hprof=cpu=samples,heap=sites,force=n,thread=y,verbose=n,file=%s","yarn.app.mapreduce.shuffle.log.backups":"0","yarn.nodemanager.container-diagnostics-maximum-size":"10000","hadoop.registry.zk.retry.interval.ms":"1000","yarn.nodemanager.linux-container-executor.cgroups.delete-timeout-ms":"1000","fs.AbstractFileSystem.file.impl":"org.apache.hadoop.fs.local.LocalFs","yarn.nodemanager.log-aggregation.roll-monitoring-interval-seconds":"-1","mapreduce.jobhistory.cleaner.interval-ms":"86400000","hadoop.registry.zk.quorum":"localhost:2181","mapreduce.output.fileoutputformat.compress":"false","yarn.resourcemanager.am-rm-tokens.master-key-rolling-interval-secs":"*********(redacted)","fs.s3a.assumed.role.session.duration":"30m","hadoop.security.group.mapping.ldap.conversion.rule":"none","hadoop.ssl.server.conf":"ssl-server.xml","fs.s3a.retry.throttle.interval":"1000ms","seq.io.sort.factor":"100","yarn.sharedcache.cleaner.initial-delay-mins":"10","mapreduce.client.completion.pollinterval":"5000","hadoop.ssl.keystores.factory.class":"org.apache.hadoop.security.ssl.FileBasedKeyStoresFactory","yarn.app.mapreduce.am.resource.cpu-vcores":"1","yarn.timeline-service.enabled":"false","yarn.nodemanager.runtime.linux.docker.capabilities":"CHOWN,DAC_OVERRIDE,FSETID,FOWNER,MKNOD,NET_RAW,SETGID,SETUID,SETFCAP,SETPCAP,NET_BIND_SERVICE,SYS_CHROOT,KILL,AUDIT_WRITE","yarn.acl.enable":"false","yarn.timeline-service.entity-group-fs-store.done-dir":"/tmp/entity-file-history/done/","mapreduce.task.profile":"false","yarn.resourcemanager.fs.state-store.uri":"${hadoop.tmp.dir}/yarn/system/rmstore","mapreduce.jobhistory.always-scan-user-dir":"false","yarn.nodemanager.opportunistic-containers-use-pause-for-preemption":"false","yarn.nodemanager.linux-container-executor.nonsecure-mode.local-user":"nobody","yarn.timeline-service.reader.class":"org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineReaderImpl","yarn.resourcemanager.configuration.provider-class":"org.apache.hadoop.yarn.LocalConfigurationProvider","yarn.nodemanager.runtime.linux.docker.userremapping-uid-threshold":"1","yarn.resourcemanager.configuration.file-system-based-store":"/yarn/conf","mapreduce.job.cache.limit.max-single-resource-mb":"0","yarn.nodemanager.runtime.linux.docker.stop.grace-period":"10","yarn.resourcemanager.resource-profiles.source-file":"resource-profiles.json","yarn.nodemanager.resource.percentage-physical-cpu-limit":"100","mapreduce.jobhistory.client.thread-count":"10","tfile.fs.input.buffer.size":"262144","mapreduce.client.progressmonitor.pollinterval":"1000","yarn.nodemanager.log-dirs":"${yarn.log.dir}/userlogs","fs.automatic.close":"true","yarn.nodemanager.hostname":"0.0.0.0","yarn.nodemanager.resource.memory.cgroups.swappiness":"0","ftp.stream-buffer-size":"4096","yarn.fail-fast":"false","yarn.timeline-service.app-aggregation-interval-secs":"15","hadoop.security.group.mapping.ldap.search.filter.user":"(&(objectClass=user)(sAMAccountName={0}))","yarn.nodemanager.container-localizer.log.level":"INFO","yarn.timeline-service.address":"${yarn.timeline-service.hostname}:10200","mapreduce.job.ubertask.maxmaps":"9","fs.s3a.threads.keepalivetime":"60","mapreduce.jobhistory.webapp.rest-csrf.methods-to-ignore":"GET,OPTIONS,HEAD","mapreduce.task.files.preserve.failedtasks":"false","yarn.app.mapreduce.client.job.retry-interval":"2000","ha.failover-controller.graceful-fence.connection.retries":"1","yarn.resourcemanager.delegation.token.max-lifetime":"*********(redacted)","yarn.timeline-service.client.drain-entities.timeout.ms":"2000","yarn.nodemanager.resource-plugins.fpga.vendor-plugin.class":"org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.fpga.IntelFpgaOpenclPlugin","yarn.timeline-service.entity-group-fs-store.summary-store":"org.apache.hadoop.yarn.server.timeline.LeveldbTimelineStore","mapreduce.reduce.cpu.vcores":"1","mapreduce.job.encrypted-intermediate-data.buffer.kb":"128","fs.client.resolve.remote.symlinks":"true","yarn.nodemanager.webapp.https.address":"0.0.0.0:8044","hadoop.http.cross-origin.allowed-origins":"*","mapreduce.job.encrypted-intermediate-data":"false","yarn.timeline-service.entity-group-fs-store.retain-seconds":"604800","yarn.resourcemanager.metrics.runtime.buckets":"60,300,1440","yarn.timeline-service.generic-application-history.max-applications":"10000","yarn.nodemanager.local-dirs":"${hadoop.tmp.dir}/nm-local-dir","mapreduce.shuffle.connection-keep-alive.enable":"false","yarn.node-labels.configuration-type":"centralized","fs.s3a.path.style.access":"false","yarn.nodemanager.aux-services.mapreduce_shuffle.class":"org.apache.hadoop.mapred.ShuffleHandler","yarn.sharedcache.store.in-memory.staleness-period-mins":"10080","fs.adl.impl":"org.apache.hadoop.fs.adl.AdlFileSystem","yarn.resourcemanager.nodemanager.minimum.version":"NONE","mapreduce.jobhistory.webapp.xfs-filter.xframe-options":"SAMEORIGIN","yarn.app.mapreduce.am.staging-dir.erasurecoding.enabled":"false","net.topology.impl":"org.apache.hadoop.net.NetworkTopology","io.map.index.skip":"0","yarn.timeline-service.reader.webapp.https.address":"${yarn.timeline-service.webapp.https.address}","fs.ftp.data.connection.mode":"ACTIVE_LOCAL_DATA_CONNECTION_MODE","mapreduce.job.local-fs.single-disk-limit.check.kill-limit-exceed":"true","yarn.scheduler.maximum-allocation-vcores":"4","hadoop.http.cross-origin.allowed-headers":"X-Requested-With,Content-Type,Accept,Origin","yarn.nodemanager.log-aggregation.compression-type":"none","yarn.timeline-service.version":"1.0f","yarn.ipc.rpc.class":"org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC","mapreduce.reduce.maxattempts":"4","hadoop.security.dns.log-slow-lookups.enabled":"false","mapreduce.job.committer.setup.cleanup.needed":"true","mapreduce.job.running.reduce.limit":"0","ipc.maximum.response.length":"134217728","yarn.resourcemanager.webapp.rest-csrf.methods-to-ignore":"GET,OPTIONS,HEAD","mapreduce.job.token.tracking.ids.enabled":"*********(redacted)","hadoop.caller.context.max.size":"128","yarn.nodemanager.runtime.linux.docker.host-pid-namespace.allowed":"false","yarn.nodemanager.runtime.linux.docker.delayed-removal.allowed":"false","hadoop.registry.system.acls":"sasl:yarn@, sasl:mapred@, sasl:hdfs@","yarn.nodemanager.recovery.dir":"${hadoop.tmp.dir}/yarn-nm-recovery","fs.s3a.fast.upload.buffer":"disk","mapreduce.jobhistory.intermediate-done-dir":"${yarn.app.mapreduce.am.staging-dir}/history/done_intermediate","yarn.app.mapreduce.shuffle.log.separate":"true","fs.s3a.max.total.tasks":"5","fs.s3a.readahead.range":"64K","hadoop.http.authentication.simple.anonymous.allowed":"true","fs.s3a.attempts.maximum":"20","hadoop.registry.zk.connection.timeout.ms":"15000","yarn.resourcemanager.delegation-token-renewer.thread-count":"*********(redacted)","yarn.nodemanager.health-checker.script.timeout-ms":"1200000","yarn.timeline-service.leveldb-timeline-store.start-time-write-cache-size":"10000","yarn.resourcemanager.resource-profiles.enabled":"false","yarn.timeline-service.hbase-schema.prefix":"prod.","fs.azure.authorization":"false","mapreduce.map.log.level":"INFO","yarn.resourcemanager.decommissioning-nodes-watcher.poll-interval-secs":"20","mapreduce.output.fileoutputformat.compress.type":"RECORD","yarn.resourcemanager.leveldb-state-store.path":"${hadoop.tmp.dir}/yarn/system/rmstore","yarn.timeline-service.webapp.rest-csrf.custom-header":"X-XSRF-Header","mapreduce.ifile.readahead.bytes":"4194304","yarn.sharedcache.app-checker.class":"org.apache.hadoop.yarn.server.sharedcachemanager.RemoteAppChecker","yarn.nodemanager.linux-container-executor.nonsecure-mode.limit-users":"true","yarn.nodemanager.resource.detect-hardware-capabilities":"false","mapreduce.cluster.acls.enabled":"false","mapreduce.job.speculative.retry-after-no-speculate":"1000","hadoop.security.group.mapping.ldap.search.group.hierarchy.levels":"0","yarn.resourcemanager.fs.state-store.retry-interval-ms":"1000","file.stream-buffer-size":"4096","yarn.resourcemanager.application-timeouts.monitor.interval-ms":"3000","mapreduce.map.output.compress.codec":"org.apache.hadoop.io.compress.DefaultCodec","mapreduce.map.speculative":"true","mapreduce.job.speculative.retry-after-speculate":"15000","yarn.nodemanager.linux-container-executor.cgroups.mount":"false","yarn.app.mapreduce.am.container.log.backups":"0","yarn.app.mapreduce.am.log.level":"INFO","mapreduce.job.reduce.slowstart.completedmaps":"0.05","yarn.timeline-service.http-authentication.type":"simple","hadoop.security.group.mapping.ldap.search.attr.group.name":"cn","yarn.nodemanager.resource-plugins.fpga.allowed-fpga-devices":"0,1","yarn.timeline-service.client.internal-timers-ttl-secs":"420","hadoop.http.logs.enabled":"true","fs.s3a.block.size":"32M","yarn.sharedcache.client-server.address":"0.0.0.0:8045","yarn.nodemanager.logaggregation.threadpool-size-max":"100","yarn.resourcemanager.hostname":"0.0.0.0","yarn.resourcemanager.delegation.key.update-interval":"86400000","mapreduce.reduce.shuffle.fetch.retry.enabled":"${yarn.nodemanager.recovery.enabled}","mapreduce.map.memory.mb":"-1","mapreduce.task.skip.start.attempts":"2","fs.AbstractFileSystem.hdfs.impl":"org.apache.hadoop.fs.Hdfs","yarn.nodemanager.disk-health-checker.enable":"true","ipc.client.tcpnodelay":"true","ipc.client.rpc-timeout.ms":"0","yarn.nodemanager.webapp.rest-csrf.methods-to-ignore":"GET,OPTIONS,HEAD","ipc.client.low-latency":"false","mapreduce.input.lineinputformat.linespermap":"1","yarn.router.interceptor.user.threadpool-size":"5","ipc.client.connect.max.retries.on.timeouts":"45","yarn.timeline-service.leveldb-timeline-store.read-cache-size":"104857600","fs.AbstractFileSystem.har.impl":"org.apache.hadoop.fs.HarFs","mapreduce.job.split.metainfo.maxsize":"10000000","yarn.am.liveness-monitor.expiry-interval-ms":"600000","yarn.resourcemanager.container-tokens.master-key-rolling-interval-secs":"*********(redacted)","yarn.timeline-service.entity-group-fs-store.app-cache-size":"10","fs.s3a.socket.recv.buffer":"8192","yarn.resourcemanager.resource-tracker.address":"${yarn.resourcemanager.hostname}:8031","yarn.nodemanager.node-labels.provider.fetch-timeout-ms":"1200000","mapreduce.job.heap.memory-mb.ratio":"0.8","yarn.resourcemanager.leveldb-state-store.compaction-interval-secs":"3600","yarn.resourcemanager.webapp.rest-csrf.custom-header":"X-XSRF-Header","yarn.scheduler.configuration.fs.path":"file://${hadoop.tmp.dir}/yarn/system/schedconf","mapreduce.client.output.filter":"FAILED","hadoop.http.filter.initializers":"org.apache.hadoop.http.lib.StaticUserWebFilter","mapreduce.reduce.memory.mb":"-1","yarn.timeline-service.hostname":"0.0.0.0","file.replication":"1","yarn.nodemanager.container-metrics.unregister-delay-ms":"10000","yarn.nodemanager.container-metrics.period-ms":"-1","mapreduce.fileoutputcommitter.task.cleanup.enabled":"false","yarn.nodemanager.log.retain-seconds":"10800","yarn.timeline-service.entity-group-fs-store.cleaner-interval-seconds":"3600","yarn.resourcemanager.keytab":"/etc/krb5.keytab","hadoop.security.group.mapping.providers.combined":"true","mapreduce.reduce.merge.inmem.threshold":"1000","yarn.timeline-service.recovery.enabled":"false","fs.azure.saskey.usecontainersaskeyforallaccess":"true","yarn.sharedcache.nm.uploader.thread-count":"20","yarn.resourcemanager.nodemanager-graceful-decommission-timeout-secs":"3600","mapreduce.shuffle.ssl.enabled":"false","yarn.timeline-service.hbase.coprocessor.app-final-value-retention-milliseconds":"259200000","fs.s3a.committer.staging.abort.pending.uploads":"true","yarn.nodemanager.opportunistic-containers-max-queue-length":"0","yarn.resourcemanager.state-store.max-completed-applications":"${yarn.resourcemanager.max-completed-applications}","mapreduce.job.speculative.minimum-allowed-tasks":"10","yarn.log-aggregation.retain-seconds":"-1","yarn.nodemanager.disk-health-checker.min-free-space-per-disk-mb":"0","mapreduce.jobhistory.max-age-ms":"604800000","hadoop.http.cross-origin.allowed-methods":"GET,POST,HEAD","yarn.resourcemanager.opportunistic-container-allocation.enabled":"false","mapreduce.jobhistory.webapp.address":"0.0.0.0:19888","hadoop.system.tags":"YARN,HDFS,NAMENODE,DATANODE,REQUIRED,SECURITY,KERBEROS,PERFORMANCE,CLIENT\n      ,SERVER,DEBUG,DEPRECATED,COMMON,OPTIONAL","yarn.log-aggregation.file-controller.TFile.class":"org.apache.hadoop.yarn.logaggregation.filecontroller.tfile.LogAggregationTFileController","yarn.client.nodemanager-connect.max-wait-ms":"180000","yarn.resourcemanager.webapp.address":"${yarn.resourcemanager.hostname}:8088","mapreduce.jobhistory.recovery.enable":"false","mapreduce.reduce.shuffle.parallelcopies":"5","fs.AbstractFileSystem.webhdfs.impl":"org.apache.hadoop.fs.WebHdfs","fs.trash.interval":"0","yarn.app.mapreduce.client.max-retries":"3","hadoop.security.authentication":"simple","mapreduce.task.profile.reduce.params":"${mapreduce.task.profile.params}","yarn.app.mapreduce.am.resource.mb":"1536","mapreduce.input.fileinputformat.list-status.num-threads":"1","yarn.nodemanager.container-executor.class":"org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor","io.mapfile.bloom.size":"1048576","yarn.timeline-service.ttl-ms":"604800000","yarn.resourcemanager.nm-container-queuing.min-queue-length":"5","yarn.nodemanager.resource.cpu-vcores":"-1","mapreduce.job.reduces":"1","fs.s3a.multipart.size":"100M","yarn.scheduler.minimum-allocation-vcores":"1","mapreduce.job.speculative.speculative-cap-total-tasks":"0.01","hadoop.ssl.client.conf":"ssl-client.xml","mapreduce.job.queuename":"default","mapreduce.job.encrypted-intermediate-data-key-size-bits":"128","fs.s3a.metadatastore.authoritative":"false","yarn.nodemanager.webapp.xfs-filter.xframe-options":"SAMEORIGIN","ha.health-monitor.sleep-after-disconnect.ms":"1000","yarn.app.mapreduce.shuffle.log.limit.kb":"0","hadoop.security.group.mapping":"org.apache.hadoop.security.JniBasedUnixGroupsMappingWithFallback","yarn.client.application-client-protocol.poll-timeout-ms":"-1","mapreduce.jobhistory.jhist.format":"binary","yarn.resourcemanager.ha.enabled":"false","hadoop.http.staticuser.user":"dr.who","mapreduce.task.exit.timeout.check-interval-ms":"20000","mapreduce.jobhistory.intermediate-user-done-dir.permissions":"770","mapreduce.task.exit.timeout":"60000","yarn.nodemanager.linux-container-executor.resources-handler.class":"org.apache.hadoop.yarn.server.nodemanager.util.DefaultLCEResourcesHandler","mapreduce.reduce.shuffle.memory.limit.percent":"0.25","yarn.resourcemanager.reservation-system.enable":"false","mapreduce.map.output.compress":"false","ha.zookeeper.acl":"world:anyone:rwcda","ipc.server.max.connections":"0","yarn.nodemanager.runtime.linux.docker.default-container-network":"host","yarn.router.webapp.address":"0.0.0.0:8089","yarn.scheduler.maximum-allocation-mb":"8192","yarn.resourcemanager.scheduler.monitor.policies":"org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy","yarn.sharedcache.cleaner.period-mins":"1440","yarn.nodemanager.resource-plugins.gpu.docker-plugin.nvidia-docker-v1.endpoint":"http://localhost:3476/v1.0/docker/cli","yarn.app.mapreduce.am.container.log.limit.kb":"0","ipc.client.connect.retry.interval":"1000","yarn.timeline-service.http-cross-origin.enabled":"false","fs.wasbs.impl":"org.apache.hadoop.fs.azure.NativeAzureFileSystem$Secure","yarn.federation.subcluster-resolver.class":"org.apache.hadoop.yarn.server.federation.resolver.DefaultSubClusterResolverImpl","yarn.resourcemanager.zk-state-store.parent-path":"/rmstore","mapreduce.jobhistory.cleaner.enable":"true","yarn.timeline-service.client.fd-flush-interval-secs":"10","hadoop.security.kms.client.encrypted.key.cache.expiry":"43200000","yarn.client.nodemanager-client-async.thread-pool-max-size":"500","mapreduce.map.maxattempts":"4","yarn.resourcemanager.nm-container-queuing.sorting-nodes-interval-ms":"1000","fs.s3a.committer.staging.tmp.path":"tmp/staging","yarn.nodemanager.sleep-delay-before-sigkill.ms":"250","yarn.resourcemanager.nm-container-queuing.min-queue-wait-time-ms":"10","mapreduce.job.end-notification.retry.attempts":"0","yarn.nodemanager.resource.count-logical-processors-as-cores":"false","hadoop.registry.zk.root":"/registry","adl.feature.ownerandgroup.enableupn":"false","yarn.resourcemanager.zk-max-znode-size.bytes":"1048576","mapreduce.job.reduce.shuffle.consumer.plugin.class":"org.apache.hadoop.mapreduce.task.reduce.Shuffle","yarn.resourcemanager.delayed.delegation-token.removal-interval-ms":"*********(redacted)","yarn.nodemanager.localizer.cache.target-size-mb":"10240","fs.s3a.committer.staging.conflict-mode":"fail","mapreduce.client.libjars.wildcard":"true","fs.s3a.committer.staging.unique-filenames":"true","yarn.nodemanager.node-attributes.provider.fetch-timeout-ms":"1200000","fs.s3a.list.version":"2","ftp.client-write-packet-size":"65536","fs.AbstractFileSystem.adl.impl":"org.apache.hadoop.fs.adl.Adl","hadoop.security.key.default.cipher":"AES/CTR/NoPadding","yarn.client.failover-retries":"0","fs.s3a.multipart.purge.age":"86400","mapreduce.job.local-fs.single-disk-limit.check.interval-ms":"5000","net.topology.node.switch.mapping.impl":"org.apache.hadoop.net.ScriptBasedMapping","yarn.nodemanager.amrmproxy.address":"0.0.0.0:8049","ipc.server.listen.queue.size":"128","map.sort.class":"org.apache.hadoop.util.QuickSort","fs.viewfs.rename.strategy":"SAME_MOUNTPOINT","hadoop.security.kms.client.authentication.retry-count":"1","fs.permissions.umask-mode":"022","fs.s3a.assumed.role.credentials.provider":"org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider","yarn.nodemanager.vmem-check-enabled":"true","yarn.nodemanager.numa-awareness.enabled":"false","yarn.nodemanager.recovery.compaction-interval-secs":"3600","yarn.app.mapreduce.client-am.ipc.max-retries":"3","yarn.federation.registry.base-dir":"yarnfederation/","mapreduce.job.max.map":"-1","mapreduce.job.local-fs.single-disk-limit.bytes":"-1","mapreduce.job.ubertask.maxreduces":"1","hadoop.security.kms.client.encrypted.key.cache.size":"500","hadoop.security.java.secure.random.algorithm":"SHA1PRNG","ha.failover-controller.cli-check.rpc-timeout.ms":"20000","mapreduce.jobhistory.jobname.limit":"50","yarn.client.nodemanager-connect.retry-interval-ms":"10000","yarn.timeline-service.state-store-class":"org.apache.hadoop.yarn.server.timeline.recovery.LeveldbTimelineStateStore","yarn.nodemanager.env-whitelist":"JAVA_HOME,HADOOP_COMMON_HOME,HADOOP_HDFS_HOME,HADOOP_CONF_DIR,CLASSPATH_PREPEND_DISTCACHE,HADOOP_YARN_HOME,HADOOP_HOME,PATH,LANG,TZ","yarn.sharedcache.nested-level":"3","yarn.timeline-service.webapp.rest-csrf.methods-to-ignore":"GET,OPTIONS,HEAD","fs.azure.user.agent.prefix":"unknown","yarn.resourcemanager.zk-delegation-token-node.split-index":"*********(redacted)","yarn.nodemanager.numa-awareness.read-topology":"false","yarn.nodemanager.webapp.address":"${yarn.nodemanager.hostname}:8042","rpc.metrics.quantile.enable":"false","yarn.registry.class":"org.apache.hadoop.registry.client.impl.FSRegistryOperationsService","mapreduce.jobhistory.admin.acl":"*","yarn.resourcemanager.system-metrics-publisher.dispatcher.pool-size":"10","yarn.scheduler.queue-placement-rules":"user-group","hadoop.http.authentication.kerberos.keytab":"${user.home}/hadoop.keytab","yarn.resourcemanager.recovery.enabled":"false","yarn.timeline-service.webapp.rest-csrf.enabled":"false"},"System Properties":{"java.io.tmpdir":"/var/folders/p_/5j1mtw1x0579b34vy4ztcbs40000gn/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"*********(redacted)","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib","user.dir":"*********(redacted)","java.library.path":"*********(redacted)/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"25.231-b11","jetty.git.hash":"ab228fde9e55e9164c738d7fa121f8ac5acd51c9","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.8.0_231-b11","java.vm.info":"mixed mode","java.ext.dirs":"*********(redacted)/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","scala.usejavacp":"true","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Los_Angeles","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.15.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","jdk.lang.Process.launchMechanism":"POSIX_SPAWN","user.country":"US","sun.jnu.encoding":"UTF-8","user.language":"en","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"terryk","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master spark://*********(redacted) --conf spark.eventLog.logStageExecutorMetrics=true --conf spark.eventLog.enabled=true --class org.apache.spark.repl.Main --name Spark shell spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_231.jdk/Contents/Home/jre","java.version":"1.8.0_231","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"*********(redacted)/audience-annotations-0.5.0.jar":"System Classpath","*********(redacted)/zstd-jni-1.4.5-2.jar":"System Classpath","*********(redacted)/hadoop-yarn-common-3.2.0.jar":"System Classpath","*********(redacted)/apache-spark/common/unsafe/target/scala-2.12/classes/":"System Classpath","*********(redacted)/kerb-identity-1.0.1.jar":"System Classpath","*********(redacted)/jersey-hk2-2.30.jar":"System Classpath","*********(redacted)/apache-spark/mllib/target/jars/*":"System Classpath","*********(redacted)/slf4j-log4j12-1.7.30.jar":"System Classpath","*********(redacted)/kerby-pkix-1.0.1.jar":"System Classpath","*********(redacted)/metrics-jmx-4.1.1.jar":"System Classpath","*********(redacted)/xz-1.5.jar":"System Classpath","*********(redacted)/jakarta.validation-api-2.0.2.jar":"System Classpath","*********(redacted)/jetty-webapp-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/metrics-core-4.1.1.jar":"System Classpath","*********(redacted)/apache-spark/graphx/target/scala-2.12/classes/":"System Classpath","*********(redacted)/metrics-graphite-4.1.1.jar":"System Classpath","*********(redacted)/jersey-server-2.30.jar":"System Classpath","*********(redacted)/jetty-jndi-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/jetty-proxy-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/parquet-jackson-1.10.1.jar":"System Classpath","*********(redacted)/commons-crypto-1.0.0.jar":"System Classpath","*********(redacted)/xbean-asm7-shaded-4.15.jar":"System Classpath","*********(redacted)/spark-kvstore_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/netty-3.10.6.Final.jar":"System Classpath","*********(redacted)/jsr305-3.0.2.jar":"System Classpath","*********(redacted)/osgi-resource-locator-1.0.3.jar":"System Classpath","*********(redacted)/kerb-server-1.0.1.jar":"System Classpath","*********(redacted)/avro-1.8.2.jar":"System Classpath","*********(redacted)/jersey-media-jaxb-2.30.jar":"System Classpath","*********(redacted)/accessors-smart-1.2.jar":"System Classpath","*********(redacted)/spark-network-shuffle_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/protobuf-java-2.5.0.jar":"System Classpath","*********(redacted)/minlog-1.3.0.jar":"System Classpath","*********(redacted)/scala-parser-combinators_2.12-1.1.2.jar":"System Classpath","*********(redacted)/jetty-continuation-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/apache-spark/sql/hive/target/scala-2.12/classes/":"System Classpath","*********(redacted)/commons-net-3.6.jar":"System Classpath","*********(redacted)/apache-spark/core/target/jars/*":"System Classpath","*********(redacted)/istack-commons-runtime-3.0.8.jar":"System Classpath","*********(redacted)/jsp-api-2.1.jar":"System Classpath","*********(redacted)/hadoop-annotations-3.2.0.jar":"System Classpath","*********(redacted)/spark-launcher_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/jetty-servlets-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/hadoop-hdfs-client-3.2.0.jar":"System Classpath","*********(redacted)/hive-storage-api-2.7.1.jar":"System Classpath","*********(redacted)/aopalliance-repackaged-2.6.1.jar":"System Classpath","*********(redacted)/kerby-xdr-1.0.1.jar":"System Classpath","*********(redacted)/orc-mapreduce-1.5.10.jar":"System Classpath","*********(redacted)/jackson-databind-2.10.0.jar":"System Classpath","*********(redacted)/spark-mllib_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/hadoop-mapreduce-client-jobclient-3.2.0.jar":"System Classpath","*********(redacted)/hk2-utils-2.6.1.jar":"System Classpath","*********(redacted)/jetty-security-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/hadoop-client-3.2.0.jar":"System Classpath","*********(redacted)/hadoop-auth-3.2.0.jar":"System Classpath","*********(redacted)/gson-2.2.4.jar":"System Classpath","*********(redacted)/htrace-core4-4.1.0-incubating.jar":"System Classpath","*********(redacted)/curator-client-2.12.0.jar":"System Classpath","*********(redacted)/kerby-util-1.0.1.jar":"System Classpath","*********(redacted)/jackson-jaxrs-base-2.9.5.jar":"System Classpath","*********(redacted)/jackson-module-paranamer-2.10.0.jar":"System Classpath","*********(redacted)/commons-beanutils-1.9.3.jar":"System Classpath","*********(redacted)/woodstox-core-5.0.3.jar":"System Classpath","*********(redacted)/arrow-memory-0.15.1.jar":"System Classpath","*********(redacted)/activation-1.1.1.jar":"System Classpath","*********(redacted)/okhttp-2.7.5.jar":"System Classpath","*********(redacted)/jackson-mapper-asl-1.9.13.jar":"System Classpath","*********(redacted)/core-1.1.2.jar":"System Classpath","*********(redacted)/jetty-plus-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/apache-spark/resource-managers/yarn/target/scala-2.12/classes/":"System Classpath","*********(redacted)/antlr4-runtime-4.7.1.jar":"System Classpath","*********(redacted)/commons-compress-1.8.1.jar":"System Classpath","*********(redacted)/spark-tags_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/orc-shims-1.5.10.jar":"System Classpath","*********(redacted)/opencsv-2.3.jar":"System Classpath","*********(redacted)/compress-lzf-1.0.3.jar":"System Classpath","*********(redacted)/jackson-core-asl-1.9.13.jar":"System Classpath","*********(redacted)/objenesis-2.5.1.jar":"System Classpath","*********(redacted)/apache-spark/conf/":"System Classpath","*********(redacted)/apache-spark/common/network-common/target/scala-2.12/classes/":"System Classpath","*********(redacted)/httpcore-4.4.4.jar":"System Classpath","*********(redacted)/jcip-annotations-1.0-1.jar":"System Classpath","*********(redacted)/apache-spark/launcher/target/scala-2.12/classes/":"System Classpath","*********(redacted)/apache-spark/resource-managers/mesos/target/scala-2.12/classes":"System Classpath","*********(redacted)/apache-spark/sql/hive-thriftserver/target/scala-2.12/classes/":"System Classpath","*********(redacted)/jetty-util-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/jetty-server-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/jetty-xml-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/scala-collection-compat_2.12-2.1.1.jar":"System Classpath","*********(redacted)/javax.servlet-api-3.1.0.jar":"System Classpath","*********(redacted)/jackson-core-2.10.0.jar":"System Classpath","*********(redacted)/jetty-client-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/netty-all-4.1.47.Final.jar":"System Classpath","*********(redacted)/arpack_combined_all-0.1.jar":"System Classpath","*********(redacted)/jersey-container-servlet-core-2.30.jar":"System Classpath","*********(redacted)/paranamer-2.8.jar":"System Classpath","*********(redacted)/spark-sql_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/hadoop-common-3.2.0.jar":"System Classpath","*********(redacted)/jersey-common-2.30.jar":"System Classpath","*********(redacted)/hk2-locator-2.6.1.jar":"System Classpath","*********(redacted)/okio-1.6.0.jar":"System Classpath","*********(redacted)/jetty-http-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/jakarta.xml.bind-api-2.3.2.jar":"System Classpath","*********(redacted)/ivy-2.4.0.jar":"System Classpath","*********(redacted)/apache-spark/mllib/target/scala-2.12/classes/":"System Classpath","*********(redacted)/scala-library-2.12.10.jar":"System Classpath","*********(redacted)/scala-xml_2.12-1.2.0.jar":"System Classpath","*********(redacted)/apache-spark/streaming/target/scala-2.12/classes/":"System Classpath","*********(redacted)/spark-repl_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/apache-spark/common/tags/target/scala-2.12/classes/":"System Classpath","*********(redacted)/kerby-config-1.0.1.jar":"System Classpath","*********(redacted)/zookeeper-3.4.14.jar":"System Classpath","*********(redacted)/janino-3.1.2.jar":"System Classpath","*********(redacted)/breeze_2.12-1.0.jar":"System Classpath","*********(redacted)/hk2-api-2.6.1.jar":"System Classpath","*********(redacted)/hadoop-mapreduce-client-common-3.2.0.jar":"System Classpath","*********(redacted)/flatbuffers-java-1.9.0.jar":"System Classpath","*********(redacted)/curator-framework-2.13.0.jar":"System Classpath","*********(redacted)/metrics-jvm-4.1.1.jar":"System Classpath","*********(redacted)/stax2-api-3.1.4.jar":"System Classpath","*********(redacted)/json-smart-2.3.jar":"System Classpath","*********(redacted)/parquet-hadoop-1.10.1.jar":"System Classpath","*********(redacted)/kerb-common-1.0.1.jar":"System Classpath","*********(redacted)/breeze-macros_2.12-1.0.jar":"System Classpath","*********(redacted)/scala-compiler-2.12.10.jar":"System Classpath","*********(redacted)/guava-14.0.1.jar":"System Classpath","*********(redacted)/jul-to-slf4j-1.7.30.jar":"System Classpath","*********(redacted)/spark-assembly_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/spark-core_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/spark-graphx_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/jakarta.ws.rs-api-2.1.6.jar":"System Classpath","*********(redacted)/spark-unsafe_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/jetty-io-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/JTransforms-3.1.jar":"System Classpath","*********(redacted)/commons-io-2.5.jar":"System Classpath","*********(redacted)/spark-catalyst_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/spire_2.12-0.17.0-M1.jar":"System Classpath","*********(redacted)/commons-cli-1.2.jar":"System Classpath","*********(redacted)/shims-0.7.45.jar":"System Classpath","*********(redacted)/spotbugs-annotations-3.1.9.jar":"System Classpath","*********(redacted)/kerb-simplekdc-1.0.1.jar":"System Classpath","*********(redacted)/commons-codec-1.11.jar":"System Classpath","*********(redacted)/jcl-over-slf4j-1.7.30.jar":"System Classpath","*********(redacted)/lz4-java-1.7.1.jar":"System Classpath","*********(redacted)/aopalliance-1.0.jar":"System Classpath","*********(redacted)/jackson-jaxrs-json-provider-2.9.5.jar":"System Classpath","*********(redacted)/json4s-jackson_2.12-3.6.6.jar":"System Classpath","*********(redacted)/commons-math3-3.4.1.jar":"System Classpath","*********(redacted)/shapeless_2.12-2.3.3.jar":"System Classpath","*********(redacted)/spark-mllib-local_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/dnsjava-2.1.7.jar":"System Classpath","*********(redacted)/unused-1.0.0.jar":"System Classpath","*********(redacted)/log4j-1.2.17.jar":"System Classpath","*********(redacted)/kerb-client-1.0.1.jar":"System Classpath","*********(redacted)/commons-lang-2.6.jar":"System Classpath","*********(redacted)/apache-spark/repl/target/scala-2.12/classes/":"System Classpath","*********(redacted)/kerb-util-1.0.1.jar":"System Classpath","*********(redacted)/slf4j-api-1.7.30.jar":"System Classpath","*********(redacted)/jackson-annotations-2.10.0.jar":"System Classpath","*********(redacted)/kerb-core-1.0.1.jar":"System Classpath","*********(redacted)/apache-spark/sql/catalyst/target/scala-2.12/classes/":"System Classpath","*********(redacted)/spire-macros_2.12-0.17.0-M1.jar":"System Classpath","*********(redacted)/apache-spark/examples/target/scala-2.12/classes/":"System Classpath","*********(redacted)/jakarta.annotation-api-1.3.5.jar":"System Classpath","*********(redacted)/json4s-scalap_2.12-3.6.6.jar":"System Classpath","*********(redacted)/apache-spark/core/target/scala-2.12/classes/":"System Classpath","*********(redacted)/arrow-format-0.15.1.jar":"System Classpath","*********(redacted)/kryo-shaded-4.0.2.jar":"System Classpath","*********(redacted)/scala-reflect-2.12.10.jar":"System Classpath","*********(redacted)/kerb-admin-1.0.1.jar":"System Classpath","*********(redacted)/hadoop-yarn-client-3.2.0.jar":"System Classpath","*********(redacted)/nimbus-jose-jwt-4.41.1.jar":"System Classpath","*********(redacted)/apache-spark/common/kvstore/target/scala-2.12/classes/":"System Classpath","*********(redacted)/oro-2.0.8.jar":"System Classpath","*********(redacted)/jakarta.inject-2.6.1.jar":"System Classpath","*********(redacted)/chill-java-0.9.5.jar":"System Classpath","*********(redacted)/cats-kernel_2.12-2.0.0-M4.jar":"System Classpath","*********(redacted)/jersey-container-servlet-2.30.jar":"System Classpath","*********(redacted)/py4j-0.10.9.jar":"System Classpath","*********(redacted)/parquet-format-2.4.0.jar":"System Classpath","*********(redacted)/apache-spark/sql/core/target/scala-2.12/classes/":"System Classpath","*********(redacted)/jline-2.14.6.jar":"System Classpath","*********(redacted)/JLargeArrays-1.5.jar":"System Classpath","*********(redacted)/kerby-asn1-1.0.1.jar":"System Classpath","*********(redacted)/jaxb-runtime-2.3.2.jar":"System Classpath","*********(redacted)/pmml-model-1.4.8.jar":"System Classpath","*********(redacted)/parquet-encoding-1.10.1.jar":"System Classpath","*********(redacted)/machinist_2.12-0.6.8.jar":"System Classpath","*********(redacted)/commons-compiler-3.1.2.jar":"System Classpath","*********(redacted)/kerb-crypto-1.0.1.jar":"System Classpath","*********(redacted)/aircompressor-0.10.jar":"System Classpath","*********(redacted)/leveldbjni-all-1.8.jar":"System Classpath","*********(redacted)/metrics-json-4.1.1.jar":"System Classpath","*********(redacted)/jackson-module-jaxb-annotations-2.9.5.jar":"System Classpath","*********(redacted)/jaxb-api-2.2.11.jar":"System Classpath","*********(redacted)/spire-util_2.12-0.17.0-M1.jar":"System Classpath","*********(redacted)/jetty-servlet-9.4.28.v20200408.jar":"System Classpath","*********(redacted)/spark-network-common_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/spark-streaming_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/json4s-core_2.12-3.6.6.jar":"System Classpath","*********(redacted)/stream-2.9.6.jar":"System Classpath","*********(redacted)/spark-sketch_2.12-3.1.0-SNAPSHOT.jar":"System Classpath","*********(redacted)/chill_2.12-0.9.5.jar":"System Classpath","*********(redacted)/commons-configuration2-2.1.1.jar":"System Classpath","*********(redacted)/univocity-parsers-2.8.3.jar":"System Classpath","*********(redacted)/parquet-common-1.10.1.jar":"System Classpath","*********(redacted)/threeten-extra-1.5.0.jar":"System Classpath","*********(redacted)/token-provider-1.0.1.jar":"System Classpath","*********(redacted)/commons-text-1.6.jar":"System Classpath","*********(redacted)/apache-spark/common/sketch/target/scala-2.12/classes/":"System Classpath","*********(redacted)/javax.inject-1.jar":"System Classpath","*********(redacted)/httpclient-4.5.2.jar":"System Classpath","*********(redacted)/avro-ipc-1.8.2.jar":"System Classpath","*********(redacted)/curator-recipes-2.13.0.jar":"System Classpath","*********(redacted)/commons-collections-3.2.2.jar":"System Classpath","*********(redacted)/jersey-client-2.30.jar":"System Classpath","*********(redacted)/snappy-java-1.1.7.5.jar":"System Classpath","*********(redacted)/algebra_2.12-2.0.0-M2.jar":"System Classpath","*********(redacted)/javassist-3.25.0-GA.jar":"System Classpath","*********(redacted)/arrow-vector-0.15.1.jar":"System Classpath","*********(redacted)/avro-mapred-1.8.2-hadoop2.jar":"System Classpath","*********(redacted)/re2j-1.1.jar":"System Classpath","*********(redacted)/parquet-column-1.10.1.jar":"System Classpath","*********(redacted)/pyrolite-4.30.jar":"System Classpath","*********(redacted)/apache-spark/common/network-shuffle/target/scala-2.12/classes/":"System Classpath","*********(redacted)/apache-spark/common/network-yarn/target/scala-2.12/classes":"System Classpath","*********(redacted)/hadoop-yarn-api-3.2.0.jar":"System Classpath","*********(redacted)/macro-compat_2.12-1.1.1.jar":"System Classpath","*********(redacted)/spire-platform_2.12-0.17.0-M1.jar":"System Classpath","*********(redacted)/commons-lang3-3.10.jar":"System Classpath","*********(redacted)/json4s-ast_2.12-3.6.6.jar":"System Classpath","*********(redacted)/RoaringBitmap-0.7.45.jar":"System Classpath","*********(redacted)/orc-core-1.5.10.jar":"System Classpath","*********(redacted)/jackson-module-scala_2.12-2.10.0.jar":"System Classpath","*********(redacted)/hadoop-mapreduce-client-core-3.2.0.jar":"System Classpath"}}
+{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"app-20200706201101-0003","Timestamp":1594091460235,"User":"terryk"}
+{"Event":"SparkListenerExecutorAdded","Timestamp":1594091463318,"Executor ID":"0","Executor Info":{"Host":"127.0.0.1","Total Cores":16,"Log Urls":{"stdout":"http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stdout","stderr":"http://127.0.0.1:8081/logPage/?appId=app-20200706201101-0003&executorId=0&logType=stderr"},"Attributes":{},"Resources":{},"Resource Profile Id":0}}
+{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"0","Host":"127.0.0.1","Port":64419},"Maximum Memory":384093388,"Timestamp":1594091463413,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0}
+{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":0,"description":"foreach at <console>:26","details":"org.apache.spark.sql.Dataset.foreach(Dataset.scala:2862)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:26)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:30)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:32)\n$line19.$read$$iw$$iw$$iw$$iw$$iw.<init>(<console>:34)\n$line19.$read$$iw$$iw$$iw$$iw.<init>(<console>:36)\n$line19.$read$$iw$$iw$$iw.<init>(<console>:38)\n$line19.$read$$iw$$iw.<init>(<console>:40)\n$line19.$read$$iw.<init>(<console>:42)\n$line19.$read.<init>(<console>:44)\n$line19.$read$.<init>(<console>:48)\n$line19.$read$.<clinit>(<console>)\n$line19.$eval$.$print$lzycompute(<console>:7)\n$line19.$eval$.$print(<console>:6)\n$line19.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:745)","physicalPlanDescription":"== Physical Plan ==\nDeserializeToObject (8)\n+- * BroadcastHashJoin Inner BuildLeft (7)\n   :- BroadcastExchange (5)\n   :  +- * BroadcastHashJoin Inner BuildRight (4)\n   :     :- * LocalTableScan (1)\n   :     +- BroadcastExchange (3)\n   :        +- LocalTableScan (2)\n   +- * LocalTableScan (6)\n\n\n(1) LocalTableScan [codegen id : 1]\nOutput [3]: [i1#10, j1#11, k1#12]\nArguments: [i1#10, j1#11, k1#12]\n\n(2) LocalTableScan\nOutput [3]: [i2#26, j2#27, k2#28]\nArguments: [i2#26, j2#27, k2#28]\n\n(3) BroadcastExchange\nInput [3]: [i2#26, j2#27, k2#28]\nArguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint))), [id=#23]\n\n(4) BroadcastHashJoin [codegen id : 1]\nLeft keys [1]: [i1#10]\nRight keys [1]: [i2#26]\nJoin condition: None\n\n(5) BroadcastExchange\nInput [6]: [i1#10, j1#11, k1#12, i2#26, j2#27, k2#28]\nArguments: HashedRelationBroadcastMode(List(cast(input[3, int, false] as bigint))), [id=#32]\n\n(6) LocalTableScan\nOutput [3]: [i3#42, j3#43, k3#44]\nArguments: [i3#42, j3#43, k3#44]\n\n(7) BroadcastHashJoin [codegen id : 2]\nLeft keys [1]: [i2#26]\nRight keys [1]: [i3#42]\nJoin condition: None\n\n(8) DeserializeToObject\nInput [9]: [i1#10, j1#11, k1#12, i2#26, j2#27, k2#28, i3#42, j3#43, k3#44]\nArguments: createexternalrow(i1#10, j1#11, k1#12.toString, i2#26, j2#27, k2#28.toString, i3#42, j3#43, k3#44.toString, StructField(i1,IntegerType,false), StructField(j1,IntegerType,false), StructField(k1,StringType,true), StructField(i2,IntegerType,false), StructField(j2,IntegerType,false), StructField(k2,StringType,true), StructField(i3,IntegerType,false), StructField(j3,IntegerType,false), StructField(k3,StringType,true)), obj#93: org.apache.spark.sql.Row\n\n","sparkPlanInfo":{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject createexternalrow(i1#10, j1#11, k1#12.toString, i2#26, j2#27, k2#28.toString, i3#42, j3#43, k3#44.toString, StructField(i1,IntegerType,false), StructField(j1,IntegerType,false), StructField(k1,StringType,true), StructField(i2,IntegerType,false), StructField(j2,IntegerType,false), StructField(k2,StringType,true), StructField(i3,IntegerType,false), StructField(j3,IntegerType,false), StructField(k3,StringType,true)), obj#93: org.apache.spark.sql.Row","children":[{"nodeName":"WholeStageCodegen (2)","simpleString":"WholeStageCodegen (2)","children":[{"nodeName":"BroadcastHashJoin","simpleString":"BroadcastHashJoin [i2#26], [i3#42], Inner, BuildLeft","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"BroadcastExchange","simpleString":"BroadcastExchange HashedRelationBroadcastMode(List(cast(input[3, int, false] as bigint))), [id=#32]","children":[{"nodeName":"WholeStageCodegen (1)","simpleString":"WholeStageCodegen (1)","children":[{"nodeName":"BroadcastHashJoin","simpleString":"BroadcastHashJoin [i1#10], [i2#26], Inner, BuildRight","children":[{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [i1#10, j1#11, k1#12]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":8,"metricType":"sum"}]},{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"BroadcastExchange","simpleString":"BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint))), [id=#23]","children":[{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [i2#26, j2#27, k2#28]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":13,"metricType":"sum"}]}],"metadata":{},"metrics":[{"name":"data size","accumulatorId":9,"metricType":"size"},{"name":"time to collect","accumulatorId":10,"metricType":"timing"},{"name":"time to build","accumulatorId":11,"metricType":"timing"},{"name":"time to broadcast","accumulatorId":12,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":7,"metricType":"sum"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":6,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"data size","accumulatorId":2,"metricType":"size"},{"name":"time to collect","accumulatorId":3,"metricType":"timing"},{"name":"time to build","accumulatorId":4,"metricType":"timing"},{"name":"time to broadcast","accumulatorId":5,"metricType":"timing"}]}],"metadata":{},"metrics":[]},{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [i3#42, j3#43, k3#44]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":14,"metricType":"sum"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":1,"metricType":"sum"}]}],"metadata":{},"metrics":[{"name":"duration","accumulatorId":0,"metricType":"timing"}]}],"metadata":{},"metrics":[]},"time":1594091478577}
+{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1594091478844,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Number of Tasks":16,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"3\",\"name\":\"LocalTableScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"3\",\"name\":\"LocalTableScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:185)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[0],"Properties":{"spark.driver.host":"localhost","spark.eventLog.enabled":"true","spark.job.interruptOnCancel":"true","spark.driver.port":"64413","spark.repl.class.uri":"spark://localhost:64413/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/p_/5j1mtw1x0579b34vy4ztcbs40000gn/T/spark-0c713088-21d7-4b35-9b52-7a4dd4d15ae1/repl-361cef81-42cd-4875-a8a6-ce9dfe55682a","spark.app.name":"Spark shell","spark.rdd.scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.ui.showConsoleProgress":"true","spark.job.description":"broadcast exchange (runId bb0234e5-4157-49a4-b40c-6d538d9f2ec8)","spark.executor.id":"driver","spark.jobGroup.id":"bb0234e5-4157-49a4-b40c-6d538d9f2ec8","spark.eventLog.logStageExecutorMetrics":"true","spark.submit.deployMode":"client","spark.master":"*********(redacted)","spark.home":"*********(redacted)","spark.sql.execution.id":"0","spark.sql.catalogImplementation":"in-memory","spark.app.id":"app-20200706201101-0003"}}
+{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Number of Tasks":16,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"3\",\"name\":\"LocalTableScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"3\",\"name\":\"LocalTableScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:185)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)","Submission Time":1594091478860,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.driver.host":"localhost","spark.eventLog.enabled":"true","spark.job.interruptOnCancel":"true","spark.driver.port":"64413","spark.repl.class.uri":"spark://localhost:64413/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/p_/5j1mtw1x0579b34vy4ztcbs40000gn/T/spark-0c713088-21d7-4b35-9b52-7a4dd4d15ae1/repl-361cef81-42cd-4875-a8a6-ce9dfe55682a","spark.app.name":"Spark shell","spark.rdd.scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.ui.showConsoleProgress":"true","spark.job.description":"broadcast exchange (runId bb0234e5-4157-49a4-b40c-6d538d9f2ec8)","spark.executor.id":"driver","spark.jobGroup.id":"bb0234e5-4157-49a4-b40c-6d538d9f2ec8","spark.eventLog.logStageExecutorMetrics":"true","spark.submit.deployMode":"client","spark.master":"*********(redacted)","spark.home":"*********(redacted)","spark.sql.execution.id":"0","spark.sql.catalogImplementation":"in-memory","spark.app.id":"app-20200706201101-0003"}}
+{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1594091479253,"Executor ID":"0","Host":"127.0.0.1","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":1594091479269,"Executor ID":"0","Host":"127.0.0.1","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":1594091479269,"Executor ID":"0","Host":"127.0.0.1","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":1594091479270,"Executor ID":"0","Host":"127.0.0.1","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":1594091479270,"Executor ID":"0","Host":"127.0.0.1","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":1594091479270,"Executor ID":"0","Host":"127.0.0.1","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":1594091479271,"Executor ID":"0","Host":"127.0.0.1","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":1594091479272,"Executor ID":"0","Host":"127.0.0.1","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":1594091479272,"Executor ID":"0","Host":"127.0.0.1","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":1594091479273,"Executor ID":"0","Host":"127.0.0.1","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":10,"Index":10,"Attempt":0,"Launch Time":1594091479273,"Executor ID":"0","Host":"127.0.0.1","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":11,"Index":11,"Attempt":0,"Launch Time":1594091479273,"Executor ID":"0","Host":"127.0.0.1","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":12,"Index":12,"Attempt":0,"Launch Time":1594091479274,"Executor ID":"0","Host":"127.0.0.1","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":13,"Index":13,"Attempt":0,"Launch Time":1594091479274,"Executor ID":"0","Host":"127.0.0.1","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":14,"Index":14,"Attempt":0,"Launch Time":1594091479274,"Executor ID":"0","Host":"127.0.0.1","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":15,"Index":15,"Attempt":0,"Launch Time":1594091479274,"Executor ID":"0","Host":"127.0.0.1","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":1594091479271,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480364,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":109,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1387,"Value":1387,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":8922000,"Value":8922000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":80,"Value":80,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":235295000,"Value":235295000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":962,"Value":962,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":962,"Executor Deserialize CPU Time":235295000,"Executor Run Time":80,"Executor CPU Time":8922000,"Peak Execution Memory":0,"Result Size":1387,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"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":1594091479272,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480367,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":218,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1391,"Value":2778,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":8860000,"Value":17782000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":81,"Value":161,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":213308000,"Value":448603000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":963,"Value":1925,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":963,"Executor Deserialize CPU Time":213308000,"Executor Run Time":81,"Executor CPU Time":8860000,"Peak Execution Memory":0,"Result Size":1391,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"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":1594091479269,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480367,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":327,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1387,"Value":4165,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":10463000,"Value":28245000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":79,"Value":240,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":228677000,"Value":677280000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":962,"Value":2887,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":962,"Executor Deserialize CPU Time":228677000,"Executor Run Time":79,"Executor CPU Time":10463000,"Peak Execution Memory":0,"Result Size":1387,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"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":1594091479274,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480367,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"4","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":436,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1390,"Value":5555,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":8782000,"Value":37027000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":83,"Value":323,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":196368000,"Value":873648000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":962,"Value":3849,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":962,"Executor Deserialize CPU Time":196368000,"Executor Run Time":83,"Executor CPU Time":8782000,"Peak Execution Memory":0,"Result Size":1390,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"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":1594091479253,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480368,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":545,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1384,"Value":6939,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":10820000,"Value":47847000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":77,"Value":400,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":221708000,"Value":1095356000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":962,"Value":4811,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":962,"Executor Deserialize CPU Time":221708000,"Executor Run Time":77,"Executor CPU Time":10820000,"Peak Execution Memory":0,"Result Size":1384,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"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":1594091479273,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480368,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"2","Value":"7","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":654,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1406,"Value":8345,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":13213000,"Value":61060000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":78,"Value":478,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":260380000,"Value":1355736000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":963,"Value":5774,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":963,"Executor Deserialize CPU Time":260380000,"Executor Run Time":78,"Executor CPU Time":13213000,"Peak Execution Memory":0,"Result Size":1406,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"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":1594091479273,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480368,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"8","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":763,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1391,"Value":9736,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":9913000,"Value":70973000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":76,"Value":554,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":210788000,"Value":1566524000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":962,"Value":6736,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":962,"Executor Deserialize CPU Time":210788000,"Executor Run Time":76,"Executor CPU Time":9913000,"Peak Execution Memory":0,"Result Size":1391,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"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":1594091479274,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480368,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"9","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":872,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1391,"Value":11127,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":8784000,"Value":79757000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":79,"Value":633,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":235620000,"Value":1802144000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":964,"Value":7700,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":964,"Executor Deserialize CPU Time":235620000,"Executor Run Time":79,"Executor CPU Time":8784000,"Peak Execution Memory":0,"Result Size":1391,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"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":1594091479274,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480369,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":981,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1391,"Value":12518,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":12053000,"Value":91810000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":86,"Value":719,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":215398000,"Value":2017542000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":963,"Value":8663,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":963,"Executor Deserialize CPU Time":215398000,"Executor Run Time":86,"Executor CPU Time":12053000,"Peak Execution Memory":0,"Result Size":1391,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"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":1594091479270,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480369,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"2","Value":"12","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":1090,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1403,"Value":13921,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":9030000,"Value":100840000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":79,"Value":798,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":196266000,"Value":2213808000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":962,"Value":9625,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":962,"Executor Deserialize CPU Time":196266000,"Executor Run Time":79,"Executor CPU Time":9030000,"Peak Execution Memory":0,"Result Size":1403,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"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":1594091479273,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480369,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"13","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":1199,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1390,"Value":15311,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":10087000,"Value":110927000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":79,"Value":877,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":195342000,"Value":2409150000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":962,"Value":10587,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":962,"Executor Deserialize CPU Time":195342000,"Executor Run Time":79,"Executor CPU Time":10087000,"Peak Execution Memory":0,"Result Size":1390,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"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":1594091479274,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480369,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"2","Value":"15","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":1308,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1411,"Value":16722,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":12920000,"Value":123847000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":86,"Value":963,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":236044000,"Value":2645194000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":961,"Value":11548,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":961,"Executor Deserialize CPU Time":236044000,"Executor Run Time":86,"Executor CPU Time":12920000,"Peak Execution Memory":0,"Result Size":1411,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"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":1594091479270,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480370,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"16","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":21,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":1417,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1430,"Value":18152,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":10478000,"Value":134325000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":77,"Value":1040,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":205925000,"Value":2851119000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":962,"Value":12510,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":962,"Executor Deserialize CPU Time":205925000,"Executor Run Time":77,"Executor CPU Time":10478000,"Peak Execution Memory":0,"Result Size":1430,"JVM GC Time":109,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"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":1594091479270,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480370,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"17","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":1526,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1387,"Value":19539,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":8972000,"Value":143297000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":87,"Value":1127,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":206247000,"Value":3057366000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":963,"Value":13473,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":963,"Executor Deserialize CPU Time":206247000,"Executor Run Time":87,"Executor CPU Time":8972000,"Peak Execution Memory":0,"Result Size":1387,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"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":1594091479272,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480370,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"2","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":21,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":1635,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1452,"Value":20991,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":20898000,"Value":164195000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":83,"Value":1210,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":389356000,"Value":3446722000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":963,"Value":14436,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":963,"Executor Deserialize CPU Time":389356000,"Executor Run Time":83,"Executor CPU Time":20898000,"Peak Execution Memory":0,"Result Size":1452,"JVM GC Time":109,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"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":1594091479269,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480370,"Failed":false,"Killed":false,"Accumulables":[{"ID":13,"Name":"number of output rows","Update":"1","Value":"20","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":20,"Name":"internal.metrics.jvmGCTime","Update":109,"Value":1744,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Update":1384,"Value":22375,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Update":8668000,"Value":172863000,"Internal":true,"Count Failed Values":true},{"ID":17,"Name":"internal.metrics.executorRunTime","Update":82,"Value":1292,"Internal":true,"Count Failed Values":true},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Update":222167000,"Value":3668889000,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Update":963,"Value":15399,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110},"Task Metrics":{"Executor Deserialize Time":963,"Executor Deserialize CPU Time":222167000,"Executor Run Time":82,"Executor CPU Time":8668000,"Peak Execution Memory":0,"Result Size":1384,"JVM GC Time":109,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"0","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":155100856,"JVMOffHeapMemory":64239224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":6964,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":6964,"OffHeapUnifiedMemory":0,"DirectPoolMemory":1852,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":7,"MinorGCTime":33,"MajorGCCount":3,"MajorGCTime":110}}
+{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Number of Tasks":16,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"3\",\"name\":\"LocalTableScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"3\",\"name\":\"LocalTableScan\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:185)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)","Submission Time":1594091478860,"Completion Time":1594091480381,"Accumulables":[{"ID":17,"Name":"internal.metrics.executorRunTime","Value":1292,"Internal":true,"Count Failed Values":true},{"ID":20,"Name":"internal.metrics.jvmGCTime","Value":1744,"Internal":true,"Count Failed Values":true},{"ID":13,"Name":"number of output rows","Value":"20","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":16,"Name":"internal.metrics.executorDeserializeCpuTime","Value":3668889000,"Internal":true,"Count Failed Values":true},{"ID":19,"Name":"internal.metrics.resultSize","Value":22375,"Internal":true,"Count Failed Values":true},{"ID":18,"Name":"internal.metrics.executorCpuTime","Value":172863000,"Internal":true,"Count Failed Values":true},{"ID":21,"Name":"internal.metrics.resultSerializationTime","Value":3,"Internal":true,"Count Failed Values":true},{"ID":15,"Name":"internal.metrics.executorDeserializeTime","Value":15399,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}}
+{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1594091480385,"Job Result":{"Result":"JobSucceeded"}}
+{"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":0,"accumUpdates":[[9,1048632],[10,1736],[11,37],[12,5]]}
+{"Event":"SparkListenerJobStart","Job ID":1,"Submission Time":1594091480498,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Number of Tasks":16,"RDD Info":[{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:185)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[1],"Properties":{"spark.driver.host":"localhost","spark.eventLog.enabled":"true","spark.job.interruptOnCancel":"true","spark.driver.port":"64413","spark.repl.class.uri":"spark://localhost:64413/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/p_/5j1mtw1x0579b34vy4ztcbs40000gn/T/spark-0c713088-21d7-4b35-9b52-7a4dd4d15ae1/repl-361cef81-42cd-4875-a8a6-ce9dfe55682a","spark.app.name":"Spark shell","spark.rdd.scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.ui.showConsoleProgress":"true","spark.job.description":"broadcast exchange (runId bac7289a-c1d8-4966-a6a3-d9f347e13a5d)","spark.executor.id":"driver","spark.jobGroup.id":"bac7289a-c1d8-4966-a6a3-d9f347e13a5d","spark.eventLog.logStageExecutorMetrics":"true","spark.submit.deployMode":"client","spark.master":"*********(redacted)","spark.home":"*********(redacted)","spark.sql.execution.id":"0","spark.sql.catalogImplementation":"in-memory","spark.app.id":"app-20200706201101-0003"}}
+{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Number of Tasks":16,"RDD Info":[{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:185)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)","Submission Time":1594091480499,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.driver.host":"localhost","spark.eventLog.enabled":"true","spark.job.interruptOnCancel":"true","spark.driver.port":"64413","spark.repl.class.uri":"spark://localhost:64413/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/p_/5j1mtw1x0579b34vy4ztcbs40000gn/T/spark-0c713088-21d7-4b35-9b52-7a4dd4d15ae1/repl-361cef81-42cd-4875-a8a6-ce9dfe55682a","spark.app.name":"Spark shell","spark.rdd.scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.ui.showConsoleProgress":"true","spark.job.description":"broadcast exchange (runId bac7289a-c1d8-4966-a6a3-d9f347e13a5d)","spark.executor.id":"driver","spark.jobGroup.id":"bac7289a-c1d8-4966-a6a3-d9f347e13a5d","spark.eventLog.logStageExecutorMetrics":"true","spark.submit.deployMode":"client","spark.master":"*********(redacted)","spark.home":"*********(redacted)","spark.sql.execution.id":"0","spark.sql.catalogImplementation":"in-memory","spark.app.id":"app-20200706201101-0003"}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1594091480502,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":1,"Attempt":0,"Launch Time":1594091480503,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":2,"Attempt":0,"Launch Time":1594091480503,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":3,"Attempt":0,"Launch Time":1594091480503,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":4,"Attempt":0,"Launch Time":1594091480503,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":5,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":6,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":7,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":8,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":9,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":10,"Attempt":0,"Launch Time":1594091480505,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":11,"Attempt":0,"Launch Time":1594091480505,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":12,"Attempt":0,"Launch Time":1594091480506,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":13,"Attempt":0,"Launch Time":1594091480506,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":14,"Attempt":0,"Launch Time":1594091480506,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":15,"Attempt":0,"Launch Time":1594091480507,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":4,"Attempt":0,"Launch Time":1594091480503,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480921,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"20","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"18","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"6","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":1016,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1966,"Value":1966,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":3116000,"Value":3116000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":378,"Value":378,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4258000,"Value":4258000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":29,"Value":29,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":29,"Executor Deserialize CPU Time":4258000,"Executor Run Time":378,"Executor CPU Time":3116000,"Peak Execution Memory":1016,"Result Size":1966,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":1,"Attempt":0,"Launch Time":1594091480503,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480921,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"40","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"36","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"12","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":2032,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1951,"Value":3917,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":2895000,"Value":6011000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":379,"Value":757,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2641000,"Value":6899000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":29,"Value":58,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":29,"Executor Deserialize CPU Time":2641000,"Executor Run Time":379,"Executor CPU Time":2895000,"Peak Execution Memory":1016,"Result Size":1951,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":5,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480923,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"60","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"54","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"18","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":3048,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1957,"Value":5874,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4164000,"Value":10175000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":380,"Value":1137,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2012000,"Value":8911000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":28,"Value":86,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":28,"Executor Deserialize CPU Time":2012000,"Executor Run Time":380,"Executor CPU Time":4164000,"Peak Execution Memory":1016,"Result Size":1957,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":9,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480923,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"80","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"72","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"24","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":4064,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1957,"Value":7831,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4285000,"Value":14460000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":380,"Value":1517,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2749000,"Value":11660000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":26,"Value":112,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":26,"Executor Deserialize CPU Time":2749000,"Executor Run Time":380,"Executor CPU Time":4285000,"Peak Execution Memory":1016,"Result Size":1957,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":7,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480924,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"100","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"21","Value":"93","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"7","Value":"31","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":5080,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2007,"Value":9838,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":16921000,"Value":31381000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":383,"Value":1900,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3854000,"Value":15514000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":26,"Value":138,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":26,"Executor Deserialize CPU Time":3854000,"Executor Run Time":383,"Executor CPU Time":16921000,"Peak Execution Memory":1016,"Result Size":2007,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":14,"Attempt":0,"Launch Time":1594091480506,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480925,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"120","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"111","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"37","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":6096,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1955,"Value":11793,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4798000,"Value":36179000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":382,"Value":2282,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2959000,"Value":18473000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":24,"Value":162,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":24,"Executor Deserialize CPU Time":2959000,"Executor Run Time":382,"Executor CPU Time":4798000,"Peak Execution Memory":1016,"Result Size":1955,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":11,"Attempt":0,"Launch Time":1594091480505,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480926,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"140","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"21","Value":"132","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"7","Value":"44","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":7112,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2015,"Value":13808,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4850000,"Value":41029000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":382,"Value":2664,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4278000,"Value":22751000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":25,"Value":187,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":25,"Executor Deserialize CPU Time":4278000,"Executor Run Time":382,"Executor CPU Time":4850000,"Peak Execution Memory":1016,"Result Size":2015,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":3,"Attempt":0,"Launch Time":1594091480503,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480927,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"21","Value":"161","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"21","Value":"153","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"7","Value":"51","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":8128,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1998,"Value":15806,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4100000,"Value":45129000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":383,"Value":3047,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3622000,"Value":26373000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":28,"Value":215,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":28,"Executor Deserialize CPU Time":3622000,"Executor Run Time":383,"Executor CPU Time":4100000,"Peak Execution Memory":1016,"Result Size":1998,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":10,"Attempt":0,"Launch Time":1594091480505,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480927,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"21","Value":"182","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"171","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"57","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":9144,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1973,"Value":17779,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4350000,"Value":49479000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":383,"Value":3430,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3471000,"Value":29844000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":25,"Value":240,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":25,"Executor Deserialize CPU Time":3471000,"Executor Run Time":383,"Executor CPU Time":4350000,"Peak Execution Memory":1016,"Result Size":1973,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":12,"Attempt":0,"Launch Time":1594091480506,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480927,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"202","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"189","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"63","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":10160,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1966,"Value":19745,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4682000,"Value":54161000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":384,"Value":3814,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2652000,"Value":32496000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":23,"Value":263,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":23,"Executor Deserialize CPU Time":2652000,"Executor Run Time":384,"Executor CPU Time":4682000,"Peak Execution Memory":1016,"Result Size":1966,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":31,"Index":15,"Attempt":0,"Launch Time":1594091480507,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480928,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"21","Value":"223","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"21","Value":"210","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"7","Value":"70","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":11176,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":2008,"Value":21753,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":3954000,"Value":58115000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":382,"Value":4196,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4289000,"Value":36785000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":25,"Value":288,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":25,"Executor Deserialize CPU Time":4289000,"Executor Run Time":382,"Executor CPU Time":3954000,"Peak Execution Memory":1016,"Result Size":2008,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":2,"Attempt":0,"Launch Time":1594091480503,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480928,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"243","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"228","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"76","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":12192,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1978,"Value":23731,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":349926000,"Value":408041000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":385,"Value":4581,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":14543000,"Value":51328000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":27,"Value":315,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":27,"Executor Deserialize CPU Time":14543000,"Executor Run Time":385,"Executor CPU Time":349926000,"Peak Execution Memory":1016,"Result Size":1978,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":6,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480928,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"263","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"246","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"82","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":13208,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1965,"Value":25696,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4473000,"Value":412514000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":381,"Value":4962,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3906000,"Value":55234000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":27,"Value":342,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":27,"Executor Deserialize CPU Time":3906000,"Executor Run Time":381,"Executor CPU Time":4473000,"Peak Execution Memory":1016,"Result Size":1965,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":13,"Attempt":0,"Launch Time":1594091480506,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480929,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"283","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"264","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"88","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":14224,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1955,"Value":27651,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":6459000,"Value":418973000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":381,"Value":5343,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3298000,"Value":58532000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":24,"Value":366,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":24,"Executor Deserialize CPU Time":3298000,"Executor Run Time":381,"Executor CPU Time":6459000,"Peak Execution Memory":1016,"Result Size":1955,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":8,"Attempt":0,"Launch Time":1594091480504,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480929,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"21","Value":"304","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"282","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"94","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":15240,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1954,"Value":29605,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":4365000,"Value":423338000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":380,"Value":5723,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2411000,"Value":60943000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":28,"Value":394,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":28,"Executor Deserialize CPU Time":2411000,"Executor Run Time":380,"Executor CPU Time":4365000,"Peak Execution Memory":1016,"Result Size":1954,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1594091480502,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091480930,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"duration","Update":"20","Value":"324","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"number of output rows","Update":"18","Value":"300","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":8,"Name":"number of output rows","Update":"6","Value":"100","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Update":1016,"Value":16256,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.resultSize","Update":1941,"Value":31546,"Internal":true,"Count Failed Values":true},{"ID":43,"Name":"internal.metrics.executorCpuTime","Update":3111000,"Value":426449000,"Internal":true,"Count Failed Values":true},{"ID":42,"Name":"internal.metrics.executorRunTime","Update":382,"Value":6105,"Internal":true,"Count Failed Values":true},{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2723000,"Value":63666000,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Update":30,"Value":424,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":30,"Executor Deserialize CPU Time":2723000,"Executor Run Time":382,"Executor CPU Time":3111000,"Peak Execution Memory":1016,"Result Size":1941,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"0","Stage ID":1,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0}}
+{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Number of Tasks":16,"RDD Info":[{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"WholeStageCodegen (1)\"}","Callsite":"$anonfun$withThreadLocalCaptured$1 at FutureTask.java:266","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.execution.SQLExecution$.$anonfun$withThreadLocalCaptured$1(SQLExecution.scala:185)\njava.util.concurrent.FutureTask.run(FutureTask.java:266)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)","Submission Time":1594091480499,"Completion Time":1594091480930,"Accumulables":[{"ID":41,"Name":"internal.metrics.executorDeserializeCpuTime","Value":63666000,"Internal":true,"Count Failed Values":true},{"ID":8,"Name":"number of output rows","Value":"100","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":44,"Name":"internal.metrics.resultSize","Value":31546,"Internal":true,"Count Failed Values":true},{"ID":49,"Name":"internal.metrics.peakExecutionMemory","Value":16256,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"internal.metrics.executorDeserializeTime","Value":424,"Internal":true,"Count Failed Values":true},{"ID":7,"Name":"number of output rows","Value":"300","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":43,"Name":"internal.metrics.executorCpuTime","Value":426449000,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"duration","Value":"324","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":42,"Name":"internal.metrics.executorRunTime","Value":6105,"Internal":true,"Count Failed Values":true}],"Resource Profile Id":0}}
+{"Event":"SparkListenerJobEnd","Job ID":1,"Completion Time":1594091480930,"Job Result":{"Result":"JobSucceeded"}}
+{"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":0,"accumUpdates":[[2,1048616],[3,2276],[4,13],[5,2]]}
+{"Event":"SparkListenerJobStart","Job ID":2,"Submission Time":1594091481039,"Stage Infos":[{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"foreach at <console>:26","Number of Tasks":16,"RDD Info":[{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"mapPartitions\"}","Callsite":"foreach at <console>:26","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"foreach at <console>:26","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"1\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"foreach at <console>:26","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"foreach at <console>:26","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"SQLExecutionRDD","Callsite":"foreach at <console>:26","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.foreach(Dataset.scala:2862)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:26)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:30)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:32)\n$line19.$read$$iw$$iw$$iw$$iw$$iw.<init>(<console>:34)\n$line19.$read$$iw$$iw$$iw$$iw.<init>(<console>:36)\n$line19.$read$$iw$$iw$$iw.<init>(<console>:38)\n$line19.$read$$iw$$iw.<init>(<console>:40)\n$line19.$read$$iw.<init>(<console>:42)\n$line19.$read.<init>(<console>:44)\n$line19.$read$.<init>(<console>:48)\n$line19.$read$.<clinit>(<console>)\n$line19.$eval$.$print$lzycompute(<console>:7)\n$line19.$eval$.$print(<console>:6)\n$line19.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:745)","Accumulables":[],"Resource Profile Id":0}],"Stage IDs":[2],"Properties":{"spark.driver.host":"localhost","spark.eventLog.enabled":"true","spark.driver.port":"64413","spark.repl.class.uri":"spark://localhost:64413/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/p_/5j1mtw1x0579b34vy4ztcbs40000gn/T/spark-0c713088-21d7-4b35-9b52-7a4dd4d15ae1/repl-361cef81-42cd-4875-a8a6-ce9dfe55682a","spark.app.name":"Spark shell","spark.rdd.scope":"{\"id\":\"19\",\"name\":\"foreach\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.ui.showConsoleProgress":"true","spark.executor.id":"driver","spark.eventLog.logStageExecutorMetrics":"true","spark.submit.deployMode":"client","spark.master":"*********(redacted)","spark.home":"*********(redacted)","spark.sql.execution.id":"0","spark.sql.catalogImplementation":"in-memory","spark.app.id":"app-20200706201101-0003"}}
+{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"foreach at <console>:26","Number of Tasks":16,"RDD Info":[{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"mapPartitions\"}","Callsite":"foreach at <console>:26","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"foreach at <console>:26","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"1\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"foreach at <console>:26","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"foreach at <console>:26","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"SQLExecutionRDD","Callsite":"foreach at <console>:26","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.foreach(Dataset.scala:2862)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:26)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:30)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:32)\n$line19.$read$$iw$$iw$$iw$$iw$$iw.<init>(<console>:34)\n$line19.$read$$iw$$iw$$iw$$iw.<init>(<console>:36)\n$line19.$read$$iw$$iw$$iw.<init>(<console>:38)\n$line19.$read$$iw$$iw.<init>(<console>:40)\n$line19.$read$$iw.<init>(<console>:42)\n$line19.$read.<init>(<console>:44)\n$line19.$read$.<init>(<console>:48)\n$line19.$read$.<clinit>(<console>)\n$line19.$eval$.$print$lzycompute(<console>:7)\n$line19.$eval$.$print(<console>:6)\n$line19.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:745)","Submission Time":1594091481040,"Accumulables":[],"Resource Profile Id":0},"Properties":{"spark.driver.host":"localhost","spark.eventLog.enabled":"true","spark.driver.port":"64413","spark.repl.class.uri":"spark://localhost:64413/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/p_/5j1mtw1x0579b34vy4ztcbs40000gn/T/spark-0c713088-21d7-4b35-9b52-7a4dd4d15ae1/repl-361cef81-42cd-4875-a8a6-ce9dfe55682a","spark.app.name":"Spark shell","spark.rdd.scope":"{\"id\":\"19\",\"name\":\"foreach\"}","spark.rdd.scope.noOverride":"true","spark.submit.pyFiles":"","spark.ui.showConsoleProgress":"true","spark.executor.id":"driver","spark.eventLog.logStageExecutorMetrics":"true","spark.submit.deployMode":"client","spark.master":"*********(redacted)","spark.home":"*********(redacted)","spark.sql.execution.id":"0","spark.sql.catalogImplementation":"in-memory","spark.app.id":"app-20200706201101-0003"}}
+{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":32,"Index":0,"Attempt":0,"Launch Time":1594091481077,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":33,"Index":1,"Attempt":0,"Launch Time":1594091481082,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":34,"Index":2,"Attempt":0,"Launch Time":1594091481087,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":35,"Index":3,"Attempt":0,"Launch Time":1594091481091,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":36,"Index":4,"Attempt":0,"Launch Time":1594091481095,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":37,"Index":5,"Attempt":0,"Launch Time":1594091481100,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":38,"Index":6,"Attempt":0,"Launch Time":1594091481104,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":39,"Index":7,"Attempt":0,"Launch Time":1594091481109,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":40,"Index":8,"Attempt":0,"Launch Time":1594091481112,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":41,"Index":9,"Attempt":0,"Launch Time":1594091481116,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":42,"Index":10,"Attempt":0,"Launch Time":1594091481120,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":43,"Index":11,"Attempt":0,"Launch Time":1594091481123,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":44,"Index":12,"Attempt":0,"Launch Time":1594091481126,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":45,"Index":13,"Attempt":0,"Launch Time":1594091481129,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":46,"Index":14,"Attempt":0,"Launch Time":1594091481132,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":47,"Index":15,"Attempt":0,"Launch Time":1594091481136,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":38,"Index":6,"Attempt":0,"Launch Time":1594091481104,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091482939,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1467","Value":"1467","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"375000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"6250","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":24040,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.resultSerializationTime","Update":11,"Value":11,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":62,"Value":62,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2158,"Value":2158,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":414110000,"Value":414110000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1498,"Value":1498,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":60358000,"Value":60358000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":255,"Value":255,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":255,"Executor Deserialize CPU Time":60358000,"Executor Run Time":1498,"Executor CPU Time":414110000,"Peak Execution Memory":24040,"Result Size":2158,"JVM GC Time":62,"Result Serialization Time":11,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":47,"Index":15,"Attempt":0,"Launch Time":1594091481136,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483014,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1563","Value":"3030","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"750000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"12500","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":48080,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":52,"Value":114,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":4273,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":1324251000,"Value":1738361000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1594,"Value":3092,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":47496000,"Value":107854000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":213,"Value":468,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":213,"Executor Deserialize CPU Time":47496000,"Executor Run Time":1594,"Executor CPU Time":1324251000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":52,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":37,"Index":5,"Attempt":0,"Launch Time":1594091481100,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483015,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1563","Value":"4593","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"1125000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"18750","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":72120,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":62,"Value":176,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":6388,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":507192000,"Value":2245553000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1596,"Value":4688,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":60890000,"Value":168744000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":256,"Value":724,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":256,"Executor Deserialize CPU Time":60890000,"Executor Run Time":1596,"Executor CPU Time":507192000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":62,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":36,"Index":4,"Attempt":0,"Launch Time":1594091481095,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483015,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1564","Value":"6157","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"1500000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"25000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":96160,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":62,"Value":238,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":8503,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":503010000,"Value":2748563000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1596,"Value":6284,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":112849000,"Value":281593000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":260,"Value":984,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":260,"Executor Deserialize CPU Time":112849000,"Executor Run Time":1596,"Executor CPU Time":503010000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":62,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":35,"Index":3,"Attempt":0,"Launch Time":1594091481091,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483016,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1566","Value":"7723","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"1875000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"31250","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":120200,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":62,"Value":300,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":10618,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":502908000,"Value":3251471000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1598,"Value":7882,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":62944000,"Value":344537000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":263,"Value":1247,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":263,"Executor Deserialize CPU Time":62944000,"Executor Run Time":1598,"Executor CPU Time":502908000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":62,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":40,"Index":8,"Attempt":0,"Launch Time":1594091481112,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483016,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1562","Value":"9285","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"2250000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"37500","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":144240,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":62,"Value":362,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":12733,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":510597000,"Value":3762068000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1595,"Value":9477,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":69760000,"Value":414297000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":246,"Value":1493,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":246,"Executor Deserialize CPU Time":69760000,"Executor Run Time":1595,"Executor CPU Time":510597000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":62,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":45,"Index":13,"Attempt":0,"Launch Time":1594091481129,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483024,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1564","Value":"10849","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"2625000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"43750","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":168280,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":62,"Value":424,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":14848,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":495138000,"Value":4257206000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1595,"Value":11072,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":54222000,"Value":468519000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":221,"Value":1714,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":221,"Executor Deserialize CPU Time":54222000,"Executor Run Time":1595,"Executor CPU Time":495138000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":62,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":39,"Index":7,"Attempt":0,"Launch Time":1594091481109,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483024,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1566","Value":"12415","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"3000000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"50000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":192320,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":62,"Value":486,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":16963,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":539451000,"Value":4796657000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1596,"Value":12668,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":64380000,"Value":532899000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":254,"Value":1968,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":254,"Executor Deserialize CPU Time":64380000,"Executor Run Time":1596,"Executor CPU Time":539451000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":62,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":33,"Index":1,"Attempt":0,"Launch Time":1594091481082,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483025,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1566","Value":"13981","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"3375000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"56250","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":216360,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":62,"Value":548,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":19078,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":519178000,"Value":5315835000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1597,"Value":14265,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":54442000,"Value":587341000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":267,"Value":2235,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":267,"Executor Deserialize CPU Time":54442000,"Executor Run Time":1597,"Executor CPU Time":519178000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":62,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":34,"Index":2,"Attempt":0,"Launch Time":1594091481087,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483026,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1574","Value":"15555","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"3750000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"62500","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":240400,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":66,"Value":614,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":21193,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":508433000,"Value":5824268000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1606,"Value":15871,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":69492000,"Value":656833000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":265,"Value":2500,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":265,"Executor Deserialize CPU Time":69492000,"Executor Run Time":1606,"Executor CPU Time":508433000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":66,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":43,"Index":11,"Attempt":0,"Launch Time":1594091481123,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483029,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1578","Value":"17133","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"4125000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"68750","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":264440,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":66,"Value":680,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":23308,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":502120000,"Value":6326388000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1609,"Value":17480,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":48849000,"Value":705682000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":225,"Value":2725,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":225,"Executor Deserialize CPU Time":48849000,"Executor Run Time":1609,"Executor CPU Time":502120000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":66,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":41,"Index":9,"Attempt":0,"Launch Time":1594091481116,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483032,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1583","Value":"18716","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"4500000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"75000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":288480,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":66,"Value":746,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":25423,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":489923000,"Value":6816311000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1614,"Value":19094,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":55787000,"Value":761469000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":240,"Value":2965,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":240,"Executor Deserialize CPU Time":55787000,"Executor Run Time":1614,"Executor CPU Time":489923000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":66,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":46,"Index":14,"Attempt":0,"Launch Time":1594091481132,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483037,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1587","Value":"20303","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"4875000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"81250","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":312520,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":66,"Value":812,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":27538,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":490927000,"Value":7307238000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1618,"Value":20712,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":51464000,"Value":812933000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":218,"Value":3183,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":218,"Executor Deserialize CPU Time":51464000,"Executor Run Time":1618,"Executor CPU Time":490927000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":66,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":32,"Index":0,"Attempt":0,"Launch Time":1594091481077,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483037,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1587","Value":"21890","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"5250000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"87500","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":336560,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":66,"Value":878,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":29653,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":496683000,"Value":7803921000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1619,"Value":22331,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":56827000,"Value":869760000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":271,"Value":3454,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":271,"Executor Deserialize CPU Time":56827000,"Executor Run Time":1619,"Executor CPU Time":496683000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":66,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":42,"Index":10,"Attempt":0,"Launch Time":1594091481120,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483043,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1593","Value":"23483","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"5625000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"93750","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":360600,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":66,"Value":944,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":31768,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":508230000,"Value":8312151000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1624,"Value":23955,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":58152000,"Value":927912000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":229,"Value":3683,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":229,"Executor Deserialize CPU Time":58152000,"Executor Run Time":1624,"Executor CPU Time":508230000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":66,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":44,"Index":12,"Attempt":0,"Launch Time":1594091481126,"Executor ID":"0","Host":"127.0.0.1","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1594091483043,"Failed":false,"Killed":false,"Accumulables":[{"ID":0,"Name":"duration","Update":"1593","Value":"25076","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":1,"Name":"number of output rows","Update":"375000","Value":"6000000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":14,"Name":"number of output rows","Update":"6250","Value":"100000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Update":24040,"Value":384640,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Update":66,"Value":1010,"Internal":true,"Count Failed Values":true},{"ID":69,"Name":"internal.metrics.resultSize","Update":2115,"Value":33883,"Internal":true,"Count Failed Values":true},{"ID":68,"Name":"internal.metrics.executorCpuTime","Update":498187000,"Value":8810338000,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.executorRunTime","Update":1624,"Value":25579,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Update":51988000,"Value":979900000,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Update":222,"Value":3905,"Internal":true,"Count Failed Values":true}]},"Task Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0},"Task Metrics":{"Executor Deserialize Time":222,"Executor Deserialize CPU Time":51988000,"Executor Run Time":1624,"Executor CPU Time":498187000,"Peak Execution Memory":24040,"Result Size":2115,"JVM GC Time":66,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}}
+{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":2,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":213367864,"JVMOffHeapMemory":189011656,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":2133349,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":2133349,"OffHeapUnifiedMemory":0,"DirectPoolMemory":282024,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":13,"MinorGCTime":115,"MajorGCCount":4,"MajorGCTime":339}}
+{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"0","Stage ID":2,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":0,"JVMOffHeapMemory":0,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":0,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":0,"OffHeapUnifiedMemory":0,"DirectPoolMemory":0,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":0,"ProcessTreeJVMRSSMemory":0,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":0,"MinorGCTime":0,"MajorGCCount":0,"MajorGCTime":0}}
+{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"foreach at <console>:26","Number of Tasks":16,"RDD Info":[{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"mapPartitions\"}","Callsite":"foreach at <console>:26","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"DeserializeToObject\"}","Callsite":"foreach at <console>:26","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"1\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"foreach at <console>:26","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"WholeStageCodegen (2)\"}","Callsite":"foreach at <console>:26","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"SQLExecutionRDD","Callsite":"foreach at <console>:26","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Barrier":false,"Number of Partitions":16,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.foreach(Dataset.scala:2862)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:26)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:30)\n$line19.$read$$iw$$iw$$iw$$iw$$iw$$iw.<init>(<console>:32)\n$line19.$read$$iw$$iw$$iw$$iw$$iw.<init>(<console>:34)\n$line19.$read$$iw$$iw$$iw$$iw.<init>(<console>:36)\n$line19.$read$$iw$$iw$$iw.<init>(<console>:38)\n$line19.$read$$iw$$iw.<init>(<console>:40)\n$line19.$read$$iw.<init>(<console>:42)\n$line19.$read.<init>(<console>:44)\n$line19.$read$.<init>(<console>:48)\n$line19.$read$.<clinit>(<console>)\n$line19.$eval$.$print$lzycompute(<console>:7)\n$line19.$eval$.$print(<console>:6)\n$line19.$eval.$print(<console>)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:745)","Submission Time":1594091481040,"Completion Time":1594091483044,"Accumulables":[{"ID":68,"Name":"internal.metrics.executorCpuTime","Value":8810338000,"Internal":true,"Count Failed Values":true},{"ID":71,"Name":"internal.metrics.resultSerializationTime","Value":11,"Internal":true,"Count Failed Values":true},{"ID":74,"Name":"internal.metrics.peakExecutionMemory","Value":384640,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.executorDeserializeTime","Value":3905,"Internal":true,"Count Failed Values":true},{"ID":14,"Name":"number of output rows","Value":"100000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":67,"Name":"internal.metrics.executorRunTime","Value":25579,"Internal":true,"Count Failed Values":true},{"ID":70,"Name":"internal.metrics.jvmGCTime","Value":1010,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"number of output rows","Value":"6000000","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":69,"Name":"internal.metrics.resultSize","Value":33883,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.executorDeserializeCpuTime","Value":979900000,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"duration","Value":"25076","Internal":true,"Count Failed Values":true,"Metadata":"sql"}],"Resource Profile Id":0}}
+{"Event":"SparkListenerJobEnd","Job ID":2,"Completion Time":1594091483044,"Job Result":{"Result":"JobSucceeded"}}
+{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":0,"time":1594091483045}
+{"Event":"SparkListenerApplicationEnd","Timestamp":1594091824231}
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 d30c1df..51e38f9 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
@@ -178,6 +178,8 @@
     "executor memory usage" -> "applications/app-20161116163331-0000/executors",
     "executor resource information" -> "applications/application_1555004656427_0144/executors",
     "multiple resource profiles" -> "applications/application_1578436911597_0052/environment",
+    "stage list with peak metrics" -> "applications/app-20200706201101-0003/stages",
+    "stage with peak metrics" -> "applications/app-20200706201101-0003/stages/2/0",
 
     "app environment" -> "applications/app-20161116163331-0000/environment",
 
diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala
index 24eb168..d5829c3 100644
--- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala
@@ -1523,14 +1523,32 @@
         assert(exec.info.id === id)
         exec.info.peakMemoryMetrics match {
           case Some(actual) =>
-            ExecutorMetricType.metricToOffset.foreach { metric =>
-              assert(actual.getMetricValue(metric._1) === metrics.getMetricValue(metric._1))
-            }
+            checkExecutorMetrics(metrics, actual)
           case _ =>
             assert(false)
         }
       }
     }
+
+    // check stage level executor metrics
+    val expectedStageValues = Map(
+      0 -> StageExecutorMetrics(
+        new ExecutorMetrics(Array(7000L, 80L, 50L, 20L, 50L, 10L, 100L, 30L,
+          80L, 40L, 9000L, 4000L, 8000L, 3000L, 7000L, 2000L)),
+        Map(
+          "1" -> new ExecutorMetrics(Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L,
+            70L, 20L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L)),
+          "2" -> new ExecutorMetrics(Array(7000L, 80L, 50L, 20L, 10L, 10L, 50L, 30L,
+            80L, 40L, 9000L, 4000L, 8000L, 3000L, 7000L, 2000L)))),
+      1 -> StageExecutorMetrics(
+        new ExecutorMetrics(Array(7000L, 70L, 25L, 40L, 60L, 30L, 70L, 60L,
+          40L, 20L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L)),
+        Map(
+          "1" -> new ExecutorMetrics(Array(7000L, 70L, 20L, 30L, 60L, 30L, 70L, 55L,
+            30L, 0L, 5000L, 3000L, 4000L, 2000L, 3000L, 1000L)),
+          "2" -> new ExecutorMetrics(Array(5500L, 40L, 25L, 40L, 10L, 30L, 35L, 60L,
+            40L, 20L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L)))))
+    checkStageExecutorMetrics(expectedStageValues)
   }
 
   test("stage executor metrics") {
@@ -1573,14 +1591,74 @@
         assert(exec.info.id === id)
         exec.info.peakMemoryMetrics match {
           case Some(actual) =>
-            ExecutorMetricType.metricToOffset.foreach { metric =>
-              assert(actual.getMetricValue(metric._1) === metrics.getMetricValue(metric._1))
-            }
+            checkExecutorMetrics(metrics, actual)
           case _ =>
             assert(false)
         }
       }
     }
+
+    // check stage level executor metrics
+    val expectedStageValues = Map(
+      0 -> StageExecutorMetrics(
+        new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 50L, 10L, 100L, 30L,
+          80L, 40L, 9000L, 4000L, 8000L, 3000L, 7000L, 2000L)),
+        Map(
+          "1" -> new ExecutorMetrics(Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L,
+            70L, 20L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L)),
+          "2" -> new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L,
+            80L, 40L, 9000L, 4000L, 8000L, 3000L, 7000L, 2000L)))),
+      1 -> StageExecutorMetrics(
+        new ExecutorMetrics(Array(7000L, 80L, 50L, 40L, 60L, 30L, 80L, 60L,
+          50L, 40L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L)),
+        Map(
+          "1" -> new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L,
+            50L, 0L, 5000L, 3000L, 4000L, 2000L, 3000L, 1000L)),
+          "2" -> new ExecutorMetrics(Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L,
+            40L, 40L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L)))))
+    checkStageExecutorMetrics(expectedStageValues)
+  }
+
+  /** expected stage executor metrics */
+  private case class StageExecutorMetrics(
+      peakExecutorMetrics: ExecutorMetrics,
+      executorMetrics: Map[String, ExecutorMetrics])
+
+  private def checkExecutorMetrics(expected: ExecutorMetrics, actual: ExecutorMetrics): Unit = {
+    ExecutorMetricType.metricToOffset.foreach { metric =>
+      assert(actual.getMetricValue(metric._1) === expected.getMetricValue(metric._1))
+    }
+  }
+
+  /** check stage level peak executor metric values, and executor peak values for each stage */
+  private def checkStageExecutorMetrics(expectedStageValues: Map[Int, StageExecutorMetrics]) = {
+    // check stage level peak executor metric values for each stage
+    for ((stageId, expectedMetrics) <- expectedStageValues) {
+      check[StageDataWrapper](Array(stageId, 0)) { stage =>
+        stage.info.peakExecutorMetrics match {
+          case Some(actual) =>
+            checkExecutorMetrics(expectedMetrics.peakExecutorMetrics, actual)
+          case None =>
+            assert(false)
+        }
+      }
+    }
+
+    // check peak executor metric values for each stage and executor
+    val stageExecSummaries = store.view(classOf[ExecutorStageSummaryWrapper]).asScala.toSeq
+    stageExecSummaries.foreach { exec =>
+      expectedStageValues.get(exec.stageId) match {
+        case Some(stageValue) =>
+          (stageValue.executorMetrics.get(exec.executorId), exec.info.peakMemoryMetrics) match {
+            case (Some(expected), Some(actual)) =>
+              checkExecutorMetrics(expected, actual)
+            case _ =>
+              assert(false)
+          }
+        case None =>
+          assert(false)
+      }
+    }
   }
 
   test("storage information on executor lost/down") {
diff --git a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala
index 3d52199..48e0d21 100644
--- a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala
@@ -93,7 +93,8 @@
         tasks = None,
         executorSummary = None,
         killedTasksSummary = Map.empty,
-        ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID
+        ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID,
+        peakExecutorMetrics = None
       )
       val taskTable = new TaskPagedTable(
         stageData,
diff --git a/dev/.rat-excludes b/dev/.rat-excludes
index 3889dc9..df1dd51 100644
--- a/dev/.rat-excludes
+++ b/dev/.rat-excludes
@@ -123,3 +123,4 @@
 GangliaReporter.java
 application_1578436911597_0052
 config.properties
+app-20200706201101-0003