diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 3ef6cba8193f..d6a359db66f4 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -20,6 +20,7 @@ package org.apache.spark.internal import java.util.concurrent.TimeUnit import org.apache.spark.launcher.SparkLauncher +import org.apache.spark.metrics.GarbageCollectionMetrics import org.apache.spark.network.util.ByteUnit import org.apache.spark.scheduler.{EventLoggingListener, SchedulingMode} import org.apache.spark.storage.{DefaultTopologyMapper, RandomBlockReplicationPolicy} @@ -114,6 +115,24 @@ package object config { .booleanConf .createWithDefault(false) + private[spark] val EVENT_LOG_GC_METRICS_YOUNG_GENERATION_GARBAGE_COLLECTORS = + ConfigBuilder("spark.eventLog.gcMetrics.youngGenerationGarbageCollectors") + .doc("Names of supported young generation garbage collector. A name usually is " + + " the return of GarbageCollectorMXBean.getName. The built-in young generation garbage " + + s"collectors are ${GarbageCollectionMetrics.YOUNG_GENERATION_BUILTIN_GARBAGE_COLLECTORS}") + .stringConf + .toSequence + .createWithDefault(GarbageCollectionMetrics.YOUNG_GENERATION_BUILTIN_GARBAGE_COLLECTORS) + + private[spark] val EVENT_LOG_GC_METRICS_OLD_GENERATION_GARBAGE_COLLECTORS = + ConfigBuilder("spark.eventLog.gcMetrics.oldGenerationGarbageCollectors") + .doc("Names of supported old generation garbage collector. A name usually is " + + "the return of GarbageCollectorMXBean.getName. The built-in old generation garbage " + + s"collectors are ${GarbageCollectionMetrics.OLD_GENERATION_BUILTIN_GARBAGE_COLLECTORS}") + .stringConf + .toSequence + .createWithDefault(GarbageCollectionMetrics.OLD_GENERATION_BUILTIN_GARBAGE_COLLECTORS) + private[spark] val EVENT_LOG_OVERWRITE = ConfigBuilder("spark.eventLog.overwrite").booleanConf.createWithDefault(false) diff --git a/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala b/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala index 704b36d3118b..6d8e9a8af7b3 100644 --- a/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala +++ b/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala @@ -19,9 +19,12 @@ package org.apache.spark.metrics import java.lang.management.{BufferPoolMXBean, ManagementFactory} import javax.management.ObjectName +import scala.collection.JavaConverters._ import scala.collection.mutable +import org.apache.spark.SparkEnv import org.apache.spark.executor.ProcfsMetricsGetter +import org.apache.spark.internal.{config, Logging} import org.apache.spark.memory.MemoryManager /** @@ -99,6 +102,63 @@ case object ProcessTreeMetrics extends ExecutorMetricType { } } +case object GarbageCollectionMetrics extends ExecutorMetricType with Logging { + private var nonBuiltInCollectors: Seq[String] = Nil + + override val names = Seq( + "MinorGCCount", + "MinorGCTime", + "MajorGCCount", + "MajorGCTime" + ) + + /* We builtin some common GC collectors which categorized as young generation and old */ + private[spark] val YOUNG_GENERATION_BUILTIN_GARBAGE_COLLECTORS = Seq( + "Copy", + "PS Scavenge", + "ParNew", + "G1 Young Generation" + ) + + private[spark] val OLD_GENERATION_BUILTIN_GARBAGE_COLLECTORS = Seq( + "MarkSweepCompact", + "PS MarkSweep", + "ConcurrentMarkSweep", + "G1 Old Generation" + ) + + private lazy val youngGenerationGarbageCollector: Seq[String] = { + SparkEnv.get.conf.get(config.EVENT_LOG_GC_METRICS_YOUNG_GENERATION_GARBAGE_COLLECTORS) + } + + private lazy val oldGenerationGarbageCollector: Seq[String] = { + SparkEnv.get.conf.get(config.EVENT_LOG_GC_METRICS_OLD_GENERATION_GARBAGE_COLLECTORS) + } + + override private[spark] def getMetricValues(memoryManager: MemoryManager): Array[Long] = { + val gcMetrics = new Array[Long](names.length) // minorCount, minorTime, majorCount, majorTime + ManagementFactory.getGarbageCollectorMXBeans.asScala.foreach { mxBean => + if (youngGenerationGarbageCollector.contains(mxBean.getName)) { + gcMetrics(0) = mxBean.getCollectionCount + gcMetrics(1) = mxBean.getCollectionTime + } else if (oldGenerationGarbageCollector.contains(mxBean.getName)) { + gcMetrics(2) = mxBean.getCollectionCount + gcMetrics(3) = mxBean.getCollectionTime + } else if (!nonBuiltInCollectors.contains(mxBean.getName)) { + nonBuiltInCollectors = mxBean.getName +: nonBuiltInCollectors + // log it when first seen + logWarning(s"To enable non-built-in garbage collector(s) " + + s"$nonBuiltInCollectors, users should configure it(them) to " + + s"${config.EVENT_LOG_GC_METRICS_YOUNG_GENERATION_GARBAGE_COLLECTORS.key} or " + + s"${config.EVENT_LOG_GC_METRICS_OLD_GENERATION_GARBAGE_COLLECTORS.key}") + } else { + // do nothing + } + } + gcMetrics + } +} + case object OnHeapExecutionMemory extends MemoryManagerExecutorMetricType( _.onHeapExecutionMemoryUsed) @@ -137,7 +197,8 @@ private[spark] object ExecutorMetricType { OffHeapUnifiedMemory, DirectPoolMemory, MappedPoolMemory, - ProcessTreeMetrics + ProcessTreeMetrics, + GarbageCollectionMetrics ) diff --git a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json index 0f0ccf9858a3..71ccedced788 100644 --- a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json @@ -1,4 +1,22 @@ [ { + "id": "application_1536831636016_59384", + "name": "Spark Pi", + "attempts": [ + { + "attemptId": "1", + "startTime": "2019-01-08T04:33:43.607GMT", + "endTime": "2019-01-08T04:33:58.745GMT", + "lastUpdated": "", + "duration": 15138, + "sparkUser": "lajin", + "completed": true, + "appSparkVersion": "3.0.0-SNAPSHOT", + "lastUpdatedEpoch": 0, + "startTimeEpoch": 1546922023607, + "endTimeEpoch": 1546922038745 + } + ] +}, { "id" : "application_1538416563558_0014", "name" : "PythonBisectingKMeansExample", "attempts" : [ { diff --git a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json index e136a35a1e3a..ad5f0ea9699d 100644 --- a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json @@ -1,4 +1,22 @@ [ { + "id": "application_1536831636016_59384", + "name": "Spark Pi", + "attempts": [ + { + "attemptId": "1", + "startTime": "2019-01-08T04:33:43.607GMT", + "endTime": "2019-01-08T04:33:58.745GMT", + "lastUpdated": "", + "duration": 15138, + "sparkUser": "lajin", + "completed": true, + "appSparkVersion": "3.0.0-SNAPSHOT", + "lastUpdatedEpoch": 0, + "startTimeEpoch": 1546922023607, + "endTimeEpoch": 1546922038745 + } + ] +}, { "id" : "application_1538416563558_0014", "name" : "PythonBisectingKMeansExample", "attempts" : [ { diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_garbage_collection_metrics_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_garbage_collection_metrics_json_expectation.json new file mode 100644 index 000000000000..f0f39e60bdd5 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_garbage_collection_metrics_json_expectation.json @@ -0,0 +1,122 @@ +[ { + "id" : "driver", + "hostPort" : "047.company.com:42509", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 0, + "maxTasks" : 0, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 0, + "totalTasks" : 0, + "totalDuration" : 0, + "totalGCTime" : 0, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : false, + "maxMemory" : 100977868, + "addTime" : "2019-01-08T04:33:44.502GMT", + "executorLogs" : { + "stdout" : "https://047.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000001/lajin/stdout?start=-4096", + "stderr" : "https://047.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000001/lajin/stderr?start=-4096" + }, + "memoryMetrics" : { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 100977868, + "totalOffHeapStorageMemory" : 0 + }, + "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "JVMHeapMemory" : 211171816, + "JVMOffHeapMemory" : 90237256, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 4876, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 4876, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 806275, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 2646888448, + "ProcessTreeJVMRSSMemory" : 520900608, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 8, + "MinorGCTime" : 374, + "MajorGCCount" : 3, + "MajorGCTime" : 170 + }, + "attributes" : { } +}, { + "id" : "2", + "hostPort" : "028.company.com:46325", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 52, + "totalTasks" : 52, + "totalDuration" : 8879, + "totalGCTime" : 420, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : false, + "maxMemory" : 97832140, + "addTime" : "2019-01-08T04:33:54.270GMT", + "executorLogs" : { + "stdout" : "https://028.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000003/lajin/stdout?start=-4096", + "stderr" : "https://028.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000003/lajin/stderr?start=-4096" + }, + "memoryMetrics" : { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 97832140, + "totalOffHeapStorageMemory" : 0 + }, + "blacklistedInStages" : [ ], + "attributes" : { } +}, { + "id" : "1", + "hostPort" : "036.company.com:35126", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 48, + "totalTasks" : 48, + "totalDuration" : 8837, + "totalGCTime" : 1192, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : false, + "maxMemory" : 97832140, + "addTime" : "2019-01-08T04:33:55.929GMT", + "executorLogs" : { + "stdout" : "https://036.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000002/lajin/stdout?start=-4096", + "stderr" : "https://036.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000002/lajin/stderr?start=-4096" + }, + "memoryMetrics" : { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 97832140, + "totalOffHeapStorageMemory" : 0 + }, + "blacklistedInStages" : [ ], + "attributes" : { } +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json index f282d483a4fb..3db537799b29 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json @@ -43,7 +43,11 @@ "ProcessTreePythonVMemory": 0, "ProcessTreePythonRSSMemory": 0, "ProcessTreeOtherVMemory": 0, - "ProcessTreeOtherRSSMemory": 0 + "ProcessTreeOtherRSSMemory": 0, + "MinorGCCount": 0, + "MinorGCTime": 0, + "MajorGCCount": 0, + "MajorGCTime": 0 }, "attributes" : { } }, { @@ -193,7 +197,11 @@ "ProcessTreePythonVMemory": 0, "ProcessTreePythonRSSMemory": 0, "ProcessTreeOtherVMemory": 0, - "ProcessTreeOtherRSSMemory": 0 + "ProcessTreeOtherRSSMemory": 0, + "MinorGCCount": 0, + "MinorGCTime": 0, + "MajorGCCount": 0, + "MajorGCTime": 0 }, "attributes" : { } }, { @@ -244,7 +252,11 @@ "ProcessTreePythonVMemory": 0, "ProcessTreePythonRSSMemory": 0, "ProcessTreeOtherVMemory": 0, - "ProcessTreeOtherRSSMemory": 0 + "ProcessTreeOtherRSSMemory": 0, + "MinorGCCount": 0, + "MinorGCTime": 0, + "MajorGCCount": 0, + "MajorGCTime": 0 }, "attributes" : { } }, { @@ -295,7 +307,11 @@ "ProcessTreePythonVMemory": 0, "ProcessTreePythonRSSMemory": 0, "ProcessTreeOtherVMemory": 0, - "ProcessTreeOtherRSSMemory": 0 + "ProcessTreeOtherRSSMemory": 0, + "MinorGCCount": 0, + "MinorGCTime": 0, + "MajorGCCount": 0, + "MajorGCTime": 0 }, "attributes" : { } }, { @@ -346,7 +362,11 @@ "ProcessTreePythonVMemory": 0, "ProcessTreePythonRSSMemory": 0, "ProcessTreeOtherVMemory": 0, - "ProcessTreeOtherRSSMemory": 0 + "ProcessTreeOtherRSSMemory": 0, + "MinorGCCount": 0, + "MinorGCTime": 0, + "MajorGCCount": 0, + "MajorGCTime": 0 }, "attributes" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_process_tree_metrics_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_process_tree_metrics_json_expectation.json index 980ec85aadad..2c2efb58341b 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_process_tree_metrics_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_process_tree_metrics_json_expectation.json @@ -43,7 +43,11 @@ "ProcessTreePythonVMemory" : 408375296, "ProcessTreePythonRSSMemory" : 40284160, "ProcessTreeOtherVMemory" : 0, - "ProcessTreeOtherRSSMemory" : 0 + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount": 0, + "MinorGCTime": 0, + "MajorGCCount": 0, + "MajorGCTime": 0 }, "attributes" : { } }, { @@ -94,7 +98,11 @@ "ProcessTreePythonVMemory" : 625926144, "ProcessTreePythonRSSMemory" : 69013504, "ProcessTreeOtherVMemory" : 0, - "ProcessTreeOtherRSSMemory" : 0 + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount": 0, + "MinorGCTime": 0, + "MajorGCCount": 0, + "MajorGCTime": 0 }, "attributes" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json index 0ef9377dcb08..c303b6c565e3 100644 --- a/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json @@ -1,4 +1,22 @@ [ { + "id": "application_1536831636016_59384", + "name": "Spark Pi", + "attempts": [ + { + "attemptId": "1", + "startTime": "2019-01-08T04:33:43.607GMT", + "endTime": "2019-01-08T04:33:58.745GMT", + "lastUpdated": "", + "duration": 15138, + "sparkUser": "lajin", + "completed": true, + "appSparkVersion": "3.0.0-SNAPSHOT", + "lastUpdatedEpoch": 0, + "startTimeEpoch": 1546922023607, + "endTimeEpoch": 1546922038745 + } + ] +}, { "id" : "application_1538416563558_0014", "name" : "PythonBisectingKMeansExample", "attempts" : [ { @@ -28,19 +46,4 @@ "startTimeEpoch" : 1524182082734, "endTimeEpoch" : 1524182189134 } ] -}, { - "id" : "application_1516285256255_0012", - "name" : "Spark shell", - "attempts" : [ { - "startTime" : "2018-01-18T18:30:35.119GMT", - "endTime" : "2018-01-18T18:38:27.938GMT", - "lastUpdated" : "", - "duration" : 472819, - "sparkUser" : "attilapiros", - "completed" : true, - "appSparkVersion" : "2.3.0-SNAPSHOT", - "lastUpdatedEpoch" : 0, - "startTimeEpoch" : 1516300235119, - "endTimeEpoch" : 1516300707938 - } ] } ] diff --git a/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json index ea9dc1b97afc..ba834d5f18d2 100644 --- a/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json @@ -1,4 +1,22 @@ [ { + "id": "application_1536831636016_59384", + "name": "Spark Pi", + "attempts": [ + { + "attemptId": "1", + "startTime": "2019-01-08T04:33:43.607GMT", + "endTime": "2019-01-08T04:33:58.745GMT", + "lastUpdated": "", + "duration": 15138, + "sparkUser": "lajin", + "completed": true, + "appSparkVersion": "3.0.0-SNAPSHOT", + "lastUpdatedEpoch": 0, + "startTimeEpoch": 1546922023607, + "endTimeEpoch": 1546922038745 + } + ] +}, { "id" : "application_1538416563558_0014", "name" : "PythonBisectingKMeansExample", "attempts" : [ { diff --git a/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json index 2a77071a9ffd..62d3544e0ba1 100644 --- a/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json @@ -1,4 +1,22 @@ [ { + "id": "application_1536831636016_59384", + "name": "Spark Pi", + "attempts": [ + { + "attemptId": "1", + "startTime": "2019-01-08T04:33:43.607GMT", + "endTime": "2019-01-08T04:33:58.745GMT", + "lastUpdated": "", + "duration": 15138, + "sparkUser": "lajin", + "completed": true, + "appSparkVersion": "3.0.0-SNAPSHOT", + "lastUpdatedEpoch": 0, + "startTimeEpoch": 1546922023607, + "endTimeEpoch": 1546922038745 + } + ] +}, { "id" : "application_1538416563558_0014", "name" : "PythonBisectingKMeansExample", "attempts" : [ { diff --git a/core/src/test/resources/spark-events/application_1536831636016_59384_1 b/core/src/test/resources/spark-events/application_1536831636016_59384_1 new file mode 100755 index 000000000000..69924afead19 --- /dev/null +++ b/core/src/test/resources/spark-events/application_1536831636016_59384_1 @@ -0,0 +1,214 @@ +{"Event":"SparkListenerLogStart","Spark Version":"3.0.0-SNAPSHOT"} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"047.company.com","Port":42509},"Maximum Memory":100977868,"Timestamp":1546922024502,"Maximum Onheap Memory":100977868,"Maximum Offheap Memory":0} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/apache/releases/jdk1.8.0_121/jre","Java Version":"1.8.0_121 (Oracle Corporation)","Scala Version":"version 2.12.8"},"Spark Properties":{"spark.history.kerberos.keytab":"/etc/security/keytabs/spark.service.keytab","spark.executor.extraJavaOptions":"-XX:MaxMetaspaceSize=512m -XX:ParallelGCThreads=3","spark.driver.host":"047.company.com","spark.history.fs.logDirectory":"hdfs://hercules-sub/spark-logs","spark.eventLog.enabled":"true","spark.ssl.historyServer.trustStore":"/etc/hadoop/truststore.jks","spark.ui.port":"0","spark.driver.port":"36796","spark.shuffle.service.enabled":"true","spark.driver.extraLibraryPath":"/apache/hadoop/lib/native/Linux-amd64-64/lib:/apache/hadoop/lib/native","spark.yarn.queue":"default","spark.history.fs.update.interval":"20s","spark.yarn.historyServer.address":"master2.company.com:50070","spark.yarn.app.id":"application_1536831636016_59384","spark.sql.function.eltOutputAsString":"true","spark.yarn.access.namenodes":"hdfs://hercules-sub,hdfs://hercules","spark.app.name":"Spark Pi","spark.scheduler.mode":"FIFO","spark.history.fs.numReplayThreads":"30","spark.eventLog.logStageExecutorMetrics.enabled":"true","spark.driver.memory":"512m","spark.executor.instances":"2","spark.history.kerberos.principal":"spark/master2.company.com@COMPANY.COM","spark.yarn.am.extraJavaOptions":"-XX:MaxMetaspaceSize=512m -XX:ParallelGCThreads=3","spark.eventLog.logStageExecutorGCMetrics.enabled":"true","spark.ssl.historyServer.port":"49670","spark.ssl.historyServer.keyStoreType":"JKS","spark.sql.function.concatBinaryAsString":"true","spark.history.fs.cleaner.enabled":"true","spark.executor.id":"driver","spark.yarn.am.memory":"4g","spark.yarn.app.container.log.dir":"/hadoop/12/yarn/log/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001","spark.driver.extraJavaOptions":"-XX:MaxMetaspaceSize=512m -XX:ParallelGCThreads=3","spark.eventLog.logStageExecutorProcessTreeMetrics.enabled":"true","spark.sql.hive.caseSensitiveInferenceMode":"NEVER_INFER","spark.submit.deployMode":"cluster","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.RM_HA_URLS":"master1.company.com:50030,master2.company.com:50030","spark.master":"yarn","spark.ssl.historyServer.keyStorePassword":"*********(redacted)","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.executor.extraLibraryPath":"/apache/hadoop/lib/native/Linux-amd64-64/lib:/apache/hadoop/lib/native","spark.executor.memory":"512m","spark.driver.extraClassPath":"/apache/hadoop/share/hadoop/common/lib/hadoop-ebay-2.7.1.2.4.2.0-258.jar:/apache/hadoop/lib/hadoop-lzo-0.6.0.2.4.2.66-4.jar","spark.eventLog.dir":"hdfs://hercules-sub/spark-logs","spark.ssl.historyServer.enabled":"true","spark.ssl.historyServer.keyStore":"/etc/hadoop/keystore.jks","spark.dynamicAllocation.enabled":"false","spark.executor.extraClassPath":"/apache/hadoop/share/hadoop/common/lib/hadoop-ebay-2.7.1.2.4.2.0-258.jar:/apache/hadoop/lib/hadoop-lzo-0.6.0.2.4.2.66-4.jar","spark.executor.cores":"4","spark.history.ui.port":"49670","spark.ssl.historyServer.trustStorePassword":"*********(redacted)","spark.ssl.historyServer.protocol":"TLSv1.2","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"master1.company.com,master2.company.com","spark.history.kerberos.enabled":"true","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"https://master1.company.com:50030/proxy/application_1536831636016_59384,https://master2.company.com:50030/proxy/application_1536831636016_59384","spark.ssl.historyServer.trustStoreType":"JKS","spark.app.id":"application_1536831636016_59384"},"System Properties":{"java.io.tmpdir":"/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/tmp","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","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":"/home/lajin","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/apache/releases/jdk1.8.0_121/jre/lib/amd64","user.dir":"/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001","java.library.path":"/apache/hadoop/lib/native/Linux-amd64-64/lib:/apache/hadoop/lib/native::/usr/java/packages/lib/amd64:/usr/lib64:/lib64:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.121-b13","jetty.git.hash":"27208684755d94a92186989f695db2d7b21ebc51","java.endorsed.dirs":"/apache/releases/jdk1.8.0_121/jre/lib/endorsed","java.runtime.version":"1.8.0_121-b13","java.vm.info":"mixed mode","java.ext.dirs":"/apache/releases/jdk1.8.0_121/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/apache/releases/jdk1.8.0_121/jre/lib/resources.jar:/apache/releases/jdk1.8.0_121/jre/lib/rt.jar:/apache/releases/jdk1.8.0_121/jre/lib/sunrsasign.jar:/apache/releases/jdk1.8.0_121/jre/lib/jsse.jar:/apache/releases/jdk1.8.0_121/jre/lib/jce.jar:/apache/releases/jdk1.8.0_121/jre/lib/charsets.jar:/apache/releases/jdk1.8.0_121/jre/lib/jfr.jar:/apache/releases/jdk1.8.0_121/jre/classes","file.encoding":"UTF-8","user.timezone":"Etc/GMT+7","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"3.10.0-514.21.2.el7.x86_64","sun.os.patch.level":"unknown","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","user.language":"en","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.print.PSPrinterJob","java.awt.graphicsenv":"sun.awt.X11GraphicsEnvironment","awt.toolkit":"sun.awt.X11.XToolkit","os.name":"Linux","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"lajin","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.yarn.ApplicationMaster --class org.apache.spark.examples.SparkPi --jar file:/filer/home/lajin/SPARK-25865_new/spark-3.0.0-SNAPSHOT-bin-SPARK-25865_new/examples/jars/spark-examples_2.12-3.0.0-SNAPSHOT.jar --arg 100 --properties-file /hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_conf__/__spark_conf__.properties","java.home":"/apache/releases/jdk1.8.0_121/jre","java.version":"1.8.0_121","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/parquet-format-2.4.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-repl_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hive-metastore-1.2.1.spark2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/arrow-memory-0.10.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jersey-client-1.9.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/netty-all-4.1.30.Final.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/azure-storage-2.2.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/stream-2.7.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/metrics-jvm-3.1.5.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-configuration-1.6.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/json4s-ast_2.12-3.5.3.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/xz-1.0.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jetty-util-6.1.26.hwx.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/javax.inject-1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/hadoop-mapreduce-examples-2.7.1.2.4.2.66-4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/orc-core-1.5.3-nohive.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/scala-xml_2.12-1.0.5.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-httpclient-3.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/gson-2.2.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jersey-json-1.9.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-applications-unmanaged-am-launcher-2.7.1.2.4.2.66-4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hive-beeline-1.2.1.spark2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hk2-locator-2.4.0-b34.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/objenesis-2.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jersey-guava-2.22.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/log4j-1.2.17.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/libthrift-0.9.3.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/avro-mapred-1.8.2-hadoop2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/jersey-server-1.9.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/pyrolite-4.13.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/activation-1.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/protobuf-java-2.5.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/xz-1.0.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jackson-core-2.2.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jetty-util-6.1.26.hwx.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/paranamer-2.3.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-mapreduce-client-core-2.7.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-mllib-local_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/snappy-java-1.0.4.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-launcher_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-io-2.4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/activation-1.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/protobuf-java-2.5.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jersey-guice-1.9.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/leveldbjni-all-1.8.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-server-sharedcachemanager-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/curator-client-2.7.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/antlr4-runtime-4.7.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-server-web-proxy-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/slf4j-api-1.7.10.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/curator-recipes-2.7.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/scala-library-2.12.8.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/hadoop-mapreduce-client-hs-plugins-2.7.1.2.4.2.66-4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/parquet-hadoop-1.10.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/htrace-core-3.1.0-incubating.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/curator-client-2.7.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-server-timeline-plugins-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/junit-4.11.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jackson-core-2.2.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/zookeeper-3.4.6.2.4.2.66-4-tests.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/httpcore-4.2.5.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/JavaEWAH-0.3.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/eigenbase-properties-1.1.5.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/guice-servlet-3.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-configuration-1.6.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/snappy-java-1.0.4.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jackson-databind-2.2.3.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-server-applicationhistoryservice-2.7.1.2.4.2.66-4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/parquet-hadoop-bundle-1.6.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/curator-framework-2.7.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-beanutils-core-1.8.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-cli-1.2.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jackson-core-asl-1.9.13.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-pool-1.5.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/netty-3.6.2.Final.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jackson-databind-2.2.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/paranamer-2.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/log4j-1.2.17.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/minlog-1.3.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/aopalliance-1.0.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/xmlenc-0.52.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/kryo-shaded-4.0.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/microsoft-windowsazure-storage-sdk-0.6.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jets3t-0.9.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jackson-xc-1.9.13.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jackson-module-scala_2.12-2.9.6.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/xz-1.5.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-sql_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-server-resourcemanager-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/netty-3.6.2.Final.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-configuration-1.6.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jaxb-impl-2.2.3-1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jackson-core-asl-1.9.13.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/javax.inject-1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/slf4j-log4j12-1.7.16.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jettison-1.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/commons-logging-1.1.3.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/netty-3.6.2.Final.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/hadoop-mapreduce-client-jobclient-2.7.1.2.4.2.66-4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/chill_2.12-0.9.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/hadoop-annotations-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jersey-core-1.9.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/shapeless_2.12-2.3.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/asm-3.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/netty-3.6.2.Final.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/commons-daemon-1.0.13.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/javassist-3.18.1-GA.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/bonecp-0.8.0.RELEASE.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/hadoop-auth-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-server-nodemanager-2.7.1.2.4.2.66-5.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/commons-io-2.4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/java-xmlbuilder-0.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-collections-3.2.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/stax-api-1.0-2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/guava-11.0.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/protobuf-java-2.5.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-codec-1.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jettison-1.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-logging-1.1.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jersey-server-1.9.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/servlet-api-2.5.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/opencsv-2.3.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-mapreduce-client-app-2.7.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/curator-framework-2.7.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-compress-1.4.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/leveldbjni-all-1.8.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/protobuf-java-2.5.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/guava-14.0.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/netty-all-4.0.23.Final.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/xmlenc-0.52.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/curator-client-2.7.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-applications-distributedshell-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/json4s-core_2.12-3.5.3.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/api-asn1-api-1.0.0-M20.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-digester-1.8.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-lang-2.6.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-yarn-server-common-2.7.4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-client-2.7.1.2.4.2.66-4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jersey-server-2.22.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/ST4-4.0.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/activation-1.1.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-collections-3.2.2.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/httpclient-4.2.5.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-configuration-1.6.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jetty-6.1.26.hwx.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jetty-util-6.1.26.hwx.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-graphx_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-server-sharedcachemanager-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-beanutils-1.7.0.jar":"System Classpath","/apache/confs/hive/conf":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/java-xmlbuilder-0.4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-common-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jsp-api-2.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/guice-3.0.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-collections-3.2.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jsch-0.1.42.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/httpcore-4.2.5.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-net-3.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/osgi-resource-locator-1.0.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jsp-api-2.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-core_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-network-common_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/orc-mapreduce-1.5.3-nohive.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-streaming_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-beanutils-core-1.8.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-lang-2.6.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/api-asn1-api-1.0.0-M20.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/breeze-macros_2.12-0.13.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hk2-utils-2.4.0-b34.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/paranamer-2.8.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/api-util-1.0.0-M20.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-server-applicationhistoryservice-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/zookeeper-3.4.6.2.4.2.66-4-tests.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-server-resourcemanager-2.7.1.2.4.2.66-4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/javolution-5.5.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-common-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/guice-3.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-common-2.7.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/commons-cli-1.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/servlet-api-2.5.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/scala-parser-combinators_2.12-1.1.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jackson-annotations-2.2.3.jar":"System Classpath","/apache/confs/hadoop/conf":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/gson-2.2.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-cli-1.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/calcite-linq4j-1.2.0-incubating.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/zookeeper-3.4.6.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/snappy-java-1.0.4.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-network-shuffle_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/hadoop-lzo-0.6.0.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/jetty-6.1.26.hwx.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/compress-lzf-1.0.3.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/guava-11.0.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/junit-4.11.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-server-nodemanager-2.7.1.2.4.2.66-5.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/hadoop-hdfs-nfs-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jackson-xc-1.9.13.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-mapreduce-client-shuffle-2.7.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/javax.ws.rs-api-2.0.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/hamcrest-core-1.3.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/javassist-3.18.1-GA.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-unsafe_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/httpcore-4.2.5.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/log4j-1.2.17.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/leveldbjni-all-1.8.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-compress-1.4.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/RoaringBitmap-0.5.11.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jackson-databind-2.2.3.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/aopalliance-1.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/flatbuffers-1.2.0-3f79e055.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jetty-6.1.26.hwx.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/stringtemplate-3.2.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/log4j-1.2.17.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-logging-1.1.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-client-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jsr305-3.0.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jackson-annotations-2.9.6.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/derby-10.12.1.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/objenesis-2.5.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/arrow-format-0.10.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jsr305-3.0.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-httpclient-3.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-codec-1.10.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-beanutils-1.7.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/jackson-core-asl-1.9.13.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/aws-java-sdk-1.7.4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/api-asn1-api-1.0.0-M20.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-lang3-3.8.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-applications-distributedshell-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/xz-1.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/gson-2.2.4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jersey-json-1.9.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-client-2.7.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/javax.annotation-api-1.2.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jta-1.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/metrics-core-3.1.5.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/apache-log4j-extras-1.2.17.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jsr305-3.0.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/log4j-1.2.17.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/microsoft-windowsazure-storage-sdk-0.6.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/janino-3.0.11.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-compress-1.8.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/jsr305-3.0.0.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-api-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-registry-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jackson-jaxrs-1.9.13.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/api-util-1.0.0-M20.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jersey-server-1.9.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-net-3.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/ivy-2.4.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/javax.inject-1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/mockito-all-1.8.5.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jersey-media-jaxb-2.22.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-registry-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/paranamer-2.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-lang-2.6.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/javax.inject-2.4.0-b34.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/hadoop-common-2.7.1.2.4.2.66-5.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/javax.inject-1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-math3-3.1.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/asm-3.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-hive-thriftserver_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/commons-compress-1.4.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jtransforms-2.4.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/chill-java-0.9.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/hadoop-mapreduce-client-hs-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jersey-core-1.9.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jersey-container-servlet-2.22.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-yarn-common-2.7.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/asm-3.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hive-exec-1.2.1.spark2.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/curator-client-2.7.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/htrace-core-3.1.0-incubating.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/asm-3.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/hadoop-hdfs-2.7.1.2.4.2.66-4-tests.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-server-web-proxy-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-digester-1.8.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/antlr-2.7.7.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-math3-3.1.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/hadoop-mapreduce-client-common-2.7.1.2.4.2.66-4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/zstd-jni-1.3.2-2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jersey-json-1.9.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-mapreduce-client-common-2.7.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/oro-2.0.8.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/calcite-core-1.2.0-incubating.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/servlet-api-2.5.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jackson-annotations-2.2.3.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jackson-core-asl-1.9.13.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jsr305-3.0.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-yarn-server-web-proxy-2.7.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/univocity-parsers-2.7.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/paranamer-2.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/xercesImpl-2.9.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/hadoop-mapreduce-client-shuffle-2.7.1.2.4.2.66-4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/arrow-vector-0.10.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/api-util-1.0.0-M20.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-io-2.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jackson-core-asl-1.9.13.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-math3-3.4.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-sketch_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-mllib_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/avro-1.7.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jpam-1.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/microsoft-windowsazure-storage-sdk-0.6.0.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-digester-1.8.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jets3t-0.9.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-hive_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/guice-servlet-3.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jetty-6.1.26.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jodd-core-3.5.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-httpclient-3.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/aircompressor-0.10.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/httpclient-4.5.6.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/snappy-java-1.0.4.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-crypto-1.0.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-collections-3.2.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/hadoop-azure-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/curator-recipes-2.7.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jersey-client-1.9.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/avro-1.8.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jline-2.14.6.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hive-jdbc-1.2.1.spark2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jackson-jaxrs-1.9.13.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jackson-jaxrs-1.9.13.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/avro-1.7.4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-applications-unmanaged-am-launcher-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/aopalliance-1.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/avro-ipc-1.8.2.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/curator-framework-2.7.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jackson-databind-2.9.6.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/json4s-scalap_2.12-3.5.3.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_conf__/__hadoop_conf__":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-server-tests-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/activation-1.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/py4j-0.10.8.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-server-timeline-plugins-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/xz-1.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/calcite-avatica-1.2.0-incubating.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/stax-api-1.0-2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/macro-compat_2.12-1.1.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-hdfs-2.7.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/core-1.1.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-beanutils-1.7.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/curator-recipes-2.7.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/parquet-encoding-1.10.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jackson-core-2.9.6.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/api-asn1-api-1.0.0-M20.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_conf__":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/parquet-jackson-1.10.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/guava-11.0.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/snappy-java-1.1.7.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/xbean-asm7-shaded-4.12.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-yarn-api-2.7.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jersey-container-servlet-core-2.22.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hk2-api-2.4.0-b34.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/datanucleus-core-3.2.10.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/objenesis-2.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-annotations-2.7.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jersey-server-1.9.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-compiler-3.0.11.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/orc-shims-1.5.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-server-common-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/spark-2.2.1-yarn-shuffle.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jackson-annotations-2.2.3.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/aopalliance-1.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/joda-time-2.9.3.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/curator-recipes-2.7.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/aopalliance-repackaged-2.4.0-b34.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/xmlenc-0.52.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-yarn_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/hadoop-hdfs-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jsp-api-2.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-auth-2.7.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/spark-2.2.1-yarn-shuffle.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-yarn-client-2.7.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/jersey-core-1.9.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/htrace-core-3.1.0-incubating.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jdo-api-3.0.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-io-2.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jul-to-slf4j-1.7.16.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/leveldbjni-all-1.8.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/jersey-server-1.9.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jetty-util-6.1.26.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/super-csv-2.2.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/slf4j-api-1.7.16.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/lz4-java-1.5.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/okio-1.4.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spire-macros_2.12-0.13.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jackson-module-paranamer-2.9.6.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/netty-3.9.9.Final.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/java-xmlbuilder-0.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-beanutils-core-1.8.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/hadoop-mapreduce-client-jobclient-2.7.1.2.4.2.66-4-tests.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/gson-2.2.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/guice-3.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jsch-0.1.42.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-logging-1.1.3.jar":"System Classpath","/apache/hadoop/share/hadoop/common/lib/hadoop-ebay-2.7.1.2.4.2.0-258.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jackson-core-2.2.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/commons-codec-1.4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-lang-2.6.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jackson-jaxrs-1.9.13.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/httpclient-4.2.5.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/avro-1.7.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/parquet-common-1.10.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/arpack_combined_all-0.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jettison-1.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/httpcore-4.4.10.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/parquet-column-1.10.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/protobuf-java-2.5.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-mapreduce-client-jobclient-2.7.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-dbcp-1.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/stax-api-1.0.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spire_2.12-0.13.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/stax-api-1.0-2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/javax.servlet-api-3.1.0.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jersey-guice-1.9.jar":"System Classpath","/apache/hadoop/lib/hadoop-lzo-0.6.0.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jackson-xc-1.9.13.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/jetty-util-6.1.26.hwx.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/htrace-core-3.1.0-incubating.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-api-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jaxb-impl-2.2.3-1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jsch-0.1.42.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/fst-2.24.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/zookeeper-3.4.6.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/antlr-runtime-3.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/jersey-guice-1.9.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-math3-3.1.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/asm-3.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/breeze_2.12-0.13.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/hadoop-mapreduce-client-app-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/leveldbjni-all-1.8.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-cli-1.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-tags_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/guice-servlet-3.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/jackson-core-asl-1.9.13.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-beanutils-core-1.8.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hppc-0.7.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jersey-client-2.22.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/validation-api-1.1.0.Final.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jsp-api-2.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/zookeeper-3.4.6.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jaxb-api-2.2.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/hadoop-nfs-2.7.1.2.4.2.66-4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jetty-sslengine-6.1.26.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/servlet-api-2.5.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/metrics-json-3.1.5.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jackson-xc-1.9.13.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/commons-io-2.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/scala-compiler-2.12.8.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/guice-3.0.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jaxb-api-2.2.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/snappy-0.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-logging-1.1.3.jar":"System Classpath","/contrib/capacity-scheduler/*.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hive-cli-1.2.1.spark2.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-net-3.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/javassist-3.18.1-GA.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-beanutils-1.7.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/libfb303-0.9.3.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jaxb-api-2.2.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/datanucleus-rdbms-3.2.9.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/okhttp-2.4.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/avro-1.7.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-codec-1.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/zookeeper-3.4.6.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jetty-6.1.26.hwx.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/slf4j-log4j12-1.7.10.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/fst-2.24.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-server-tests-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jaxb-api-2.2.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/hamcrest-core-1.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jets3t-0.9.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-io-2.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/json4s-jackson_2.12-3.5.3.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-catalyst_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/jersey-core-1.9.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/metrics-graphite-3.1.5.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/api-util-1.0.0-M20.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jaxb-impl-2.2.3-1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/mysql-connector-java-5.1.40-bin.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jersey-core-1.9.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/scala-reflect-2.12.8.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/curator-framework-2.7.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/xmlenc-0.52.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-httpclient-3.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/htrace-core-3.1.0-incubating.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/xml-apis-1.3.04.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/xmlenc-0.52.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-digester-1.8.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jersey-common-2.22.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/xercesImpl-2.9.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/hadoop-common-2.7.1.2.4.2.66-4-tests.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/guava-11.0.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/protobuf-java-2.5.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-codec-1.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/machinist_2.12-0.6.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-kvstore_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/hadoop-ebay-2.7.1.2.4.2.0-258.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/log4j-1.2.17.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/hadoop-aws-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-server-common-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/netty-3.6.2.Final.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-compress-1.4.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/httpclient-4.2.5.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/guice-servlet-3.0.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/stax-api-1.0-2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/hadoop-annotations-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-cli-1.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jcl-over-slf4j-1.7.16.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/commons-lang-2.6.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-net-3.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/hadoop-mapreduce-client-core-2.7.1.2.4.2.66-4.jar":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Spark Pi","App ID":"application_1536831636016_59384","Timestamp":1546922023607,"User":"lajin","App Attempt ID":"1","Driver Logs":{"stdout":"https://047.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000001/lajin/stdout?start=-4096","stderr":"https://047.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000001/lajin/stderr?start=-4096"}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1546922034270,"Executor ID":"2","Executor Info":{"Host":"028.company.com","Total Cores":4,"Log Urls":{"stdout":"https://028.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000003/lajin/stdout?start=-4096","stderr":"https://028.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000003/lajin/stderr?start=-4096"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"2","Host":"028.company.com","Port":46325},"Maximum Memory":97832140,"Timestamp":1546922034365,"Maximum Onheap Memory":97832140,"Maximum Offheap Memory":0} +{"Event":"SparkListenerExecutorAdded","Timestamp":1546922035929,"Executor ID":"1","Executor Info":{"Host":"036.company.com","Total Cores":4,"Log Urls":{"stdout":"https://036.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000002/lajin/stdout?start=-4096","stderr":"https://036.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000002/lajin/stderr?start=-4096"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"036.company.com","Port":35126},"Maximum Memory":97832140,"Timestamp":1546922036120,"Maximum Onheap Memory":97832140,"Maximum Offheap Memory":0} +{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1546922036359,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"reduce at SparkPi.scala:38","Number of Tasks":100,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"map\"}","Callsite":"map at SparkPi.scala:34","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":100,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at SparkPi.scala:34","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":100,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.reduce(RDD.scala:1031)\norg.apache.spark.examples.SparkPi$.main(SparkPi.scala:38)\norg.apache.spark.examples.SparkPi.main(SparkPi.scala)\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)\norg.apache.spark.deploy.yarn.ApplicationMaster$$anon$2.run(ApplicationMaster.scala:677)","Accumulables":[]}],"Stage IDs":[0],"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"2\",\"name\":\"reduce\"}"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"reduce at SparkPi.scala:38","Number of Tasks":100,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"map\"}","Callsite":"map at SparkPi.scala:34","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":100,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at SparkPi.scala:34","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":100,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.reduce(RDD.scala:1031)\norg.apache.spark.examples.SparkPi$.main(SparkPi.scala:38)\norg.apache.spark.examples.SparkPi.main(SparkPi.scala)\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)\norg.apache.spark.deploy.yarn.ApplicationMaster$$anon$2.run(ApplicationMaster.scala:677)","Submission Time":1546922036383,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"2\",\"name\":\"reduce\"}"}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1546922036494,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1546922036515,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1546922036515,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1546922036515,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1546922036516,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1546922036516,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1546922036516,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1546922036517,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1546922037414,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1546922037416,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1546922036516,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037421,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":86,"Value":86,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":834,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":114738938,"Value":114738938,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":134,"Value":134,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":296991078,"Value":296991078,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":725,"Value":725,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":725,"Executor Deserialize CPU Time":296991078,"Executor Run Time":134,"Executor CPU Time":114738938,"Result Size":834,"JVM GC Time":86,"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":1546922036517,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037424,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":86,"Value":172,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":1668,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":116821463,"Value":231560401,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":134,"Value":268,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":250458033,"Value":547449111,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":725,"Value":1450,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":725,"Executor Deserialize CPU Time":250458033,"Executor Run Time":134,"Executor CPU Time":116821463,"Result Size":834,"JVM GC Time":86,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":10,"Attempt":0,"Launch Time":1546922037431,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1546922036515,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037432,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":93,"Value":265,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":2502,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":139987945,"Value":371548346,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":165,"Value":433,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":443768774,"Value":991217885,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":725,"Value":2175,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":725,"Executor Deserialize CPU Time":443768774,"Executor Run Time":165,"Executor CPU Time":139987945,"Result Size":834,"JVM GC Time":93,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":11,"Attempt":0,"Launch Time":1546922037432,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1546922036515,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037433,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":93,"Value":358,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":3336,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":143148800,"Value":514697146,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":165,"Value":598,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":257929339,"Value":1249147224,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":725,"Value":2900,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":725,"Executor Deserialize CPU Time":257929339,"Executor Run Time":165,"Executor CPU Time":143148800,"Result Size":834,"JVM GC Time":93,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":12,"Attempt":0,"Launch Time":1546922037499,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1546922037414,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037500,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":7,"Value":365,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":4170,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":68955072,"Value":583652218,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":69,"Value":667,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2463324,"Value":1251610548,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":10,"Value":2910,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":10,"Executor Deserialize CPU Time":2463324,"Executor Run Time":69,"Executor CPU Time":68955072,"Result Size":834,"JVM GC Time":7,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":13,"Attempt":0,"Launch Time":1546922037500,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1546922037416,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037501,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":7,"Value":372,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":5004,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":69409062,"Value":653061280,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":77,"Value":744,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2197208,"Value":1253807756,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":2912,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":2197208,"Executor Run Time":77,"Executor CPU Time":69409062,"Result Size":834,"JVM GC Time":7,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":14,"Attempt":0,"Launch Time":1546922037546,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":12,"Attempt":0,"Launch Time":1546922037499,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037547,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":5795,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":41068546,"Value":694129826,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":41,"Value":785,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1546605,"Value":1255354361,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":2914,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1546605,"Executor Run Time":41,"Executor CPU Time":41068546,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":15,"Attempt":0,"Launch Time":1546922037583,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1546922036516,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037584,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":280,"Value":652,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":6629,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":121468700,"Value":815598526,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":136,"Value":921,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":239941709,"Value":1495296070,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":900,"Value":3814,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":900,"Executor Deserialize CPU Time":239941709,"Executor Run Time":136,"Executor CPU Time":121468700,"Result Size":834,"JVM GC Time":280,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":16,"Attempt":0,"Launch Time":1546922037587,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":14,"Attempt":0,"Launch Time":1546922037546,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037588,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":7420,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":34831594,"Value":850430120,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":35,"Value":956,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2083011,"Value":1497379081,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":3816,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":2083011,"Executor Run Time":35,"Executor CPU Time":34831594,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":17,"Attempt":0,"Launch Time":1546922037624,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":16,"Attempt":0,"Launch Time":1546922037587,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037624,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":8211,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":31573348,"Value":882003468,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":32,"Value":988,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1326191,"Value":1498705272,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":3817,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1326191,"Executor Run Time":32,"Executor CPU Time":31573348,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":18,"Attempt":0,"Launch Time":1546922037633,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1546922036494,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037634,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":286,"Value":938,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":9045,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":174111771,"Value":1056115239,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":194,"Value":1182,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":414632138,"Value":1913337410,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":900,"Value":4717,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":900,"Executor Deserialize CPU Time":414632138,"Executor Run Time":194,"Executor CPU Time":174111771,"Result Size":834,"JVM GC Time":286,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":19,"Attempt":0,"Launch Time":1546922037639,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":15,"Attempt":0,"Launch Time":1546922037583,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037639,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":944,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":9879,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":40168837,"Value":1096284076,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":46,"Value":1228,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2240849,"Value":1915578259,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":4720,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":2240849,"Executor Run Time":46,"Executor CPU Time":40168837,"Result Size":834,"JVM GC Time":6,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":20,"Attempt":0,"Launch Time":1546922037641,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1546922036515,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037641,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":286,"Value":1230,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":10713,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":189780021,"Value":1286064097,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":203,"Value":1431,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":265791948,"Value":2181370207,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":900,"Value":5620,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":900,"Executor Deserialize CPU Time":265791948,"Executor Run Time":203,"Executor CPU Time":189780021,"Result Size":834,"JVM GC Time":286,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":21,"Attempt":0,"Launch Time":1546922037642,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1546922036516,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037642,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":286,"Value":1516,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":11547,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":184630125,"Value":1470694222,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":203,"Value":1634,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":241252253,"Value":2422622460,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":900,"Value":6520,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":900,"Executor Deserialize CPU Time":241252253,"Executor Run Time":203,"Executor CPU Time":184630125,"Result Size":834,"JVM GC Time":286,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":22,"Attempt":0,"Launch Time":1546922037662,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":17,"Attempt":0,"Launch Time":1546922037624,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037662,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":12338,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":31605508,"Value":1502299730,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":31,"Value":1665,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1157338,"Value":2423779798,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6522,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1157338,"Executor Run Time":31,"Executor CPU Time":31605508,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":23,"Attempt":0,"Launch Time":1546922037665,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":10,"Attempt":0,"Launch Time":1546922037431,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037665,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":13129,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":227941261,"Value":1730240991,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":228,"Value":1893,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1632598,"Value":2425412396,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6524,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1632598,"Executor Run Time":228,"Executor CPU Time":227941261,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":24,"Attempt":0,"Launch Time":1546922037671,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":11,"Attempt":0,"Launch Time":1546922037432,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037672,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":13920,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":234080439,"Value":1964321430,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":234,"Value":2127,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1842114,"Value":2427254510,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6526,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1842114,"Executor Run Time":234,"Executor CPU Time":234080439,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":25,"Attempt":0,"Launch Time":1546922037685,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":21,"Attempt":0,"Launch Time":1546922037642,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037685,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":14711,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":36307061,"Value":2000628491,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":37,"Value":2164,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2588273,"Value":2429842783,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6528,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":2588273,"Executor Run Time":37,"Executor CPU Time":36307061,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":26,"Attempt":0,"Launch Time":1546922037699,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":22,"Attempt":0,"Launch Time":1546922037662,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037700,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":15502,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":32277689,"Value":2032906180,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":32,"Value":2196,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1148300,"Value":2430991083,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6530,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1148300,"Executor Run Time":32,"Executor CPU Time":32277689,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":27,"Attempt":0,"Launch Time":1546922037724,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":25,"Attempt":0,"Launch Time":1546922037685,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037724,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":16293,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":32762038,"Value":2065668218,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":33,"Value":2229,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1655404,"Value":2432646487,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6532,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1655404,"Executor Run Time":33,"Executor CPU Time":32762038,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":28,"Attempt":0,"Launch Time":1546922037740,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":26,"Attempt":0,"Launch Time":1546922037699,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037740,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":17084,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":30991307,"Value":2096659525,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":31,"Value":2260,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2219464,"Value":2434865951,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6534,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":2219464,"Executor Run Time":31,"Executor CPU Time":30991307,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":29,"Attempt":0,"Launch Time":1546922037744,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":18,"Attempt":0,"Launch Time":1546922037633,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037745,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":17875,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":104659378,"Value":2201318903,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":104,"Value":2364,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1617759,"Value":2436483710,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":6537,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":1617759,"Executor Run Time":104,"Executor CPU Time":104659378,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":30,"Attempt":0,"Launch Time":1546922037758,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":27,"Attempt":0,"Launch Time":1546922037724,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037759,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":18666,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":29044222,"Value":2230363125,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":30,"Value":2394,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2017183,"Value":2438500893,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6539,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":2017183,"Executor Run Time":30,"Executor CPU Time":29044222,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":31,"Attempt":0,"Launch Time":1546922037779,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":28,"Attempt":0,"Launch Time":1546922037740,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037780,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":19457,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":34945007,"Value":2265308132,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":35,"Value":2429,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":998897,"Value":2439499790,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6540,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":998897,"Executor Run Time":35,"Executor CPU Time":34945007,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":32,"Index":32,"Attempt":0,"Launch Time":1546922037781,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":13,"Attempt":0,"Launch Time":1546922037500,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037781,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":20248,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":275257849,"Value":2540565981,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":275,"Value":2704,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1522428,"Value":2441022218,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6542,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1522428,"Executor Run Time":275,"Executor CPU Time":275257849,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":33,"Index":33,"Attempt":0,"Launch Time":1546922037791,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":30,"Attempt":0,"Launch Time":1546922037758,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037792,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":20996,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":28454753,"Value":2569020734,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":30,"Value":2734,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1143042,"Value":2442165260,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":1143042,"Executor Run Time":30,"Executor CPU Time":28454753,"Result Size":748,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":34,"Index":34,"Attempt":0,"Launch Time":1546922037822,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":32,"Index":32,"Attempt":0,"Launch Time":1546922037781,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037822,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":21830,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":36300767,"Value":2605321501,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":35,"Value":2769,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2138947,"Value":2444304207,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":6545,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":2138947,"Executor Run Time":35,"Executor CPU Time":36300767,"Result Size":834,"JVM GC Time":0,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":35,"Index":35,"Attempt":0,"Launch Time":1546922037825,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":33,"Index":33,"Attempt":0,"Launch Time":1546922037791,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037825,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":22621,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":28452489,"Value":2633773990,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":28,"Value":2797,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1182965,"Value":2445487172,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6547,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1182965,"Executor Run Time":28,"Executor CPU Time":28452489,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":36,"Index":36,"Attempt":0,"Launch Time":1546922037857,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":35,"Index":35,"Attempt":0,"Launch Time":1546922037825,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037858,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":23412,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":28114352,"Value":2661888342,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":29,"Value":2826,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1597988,"Value":2447085160,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6548,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1597988,"Executor Run Time":29,"Executor CPU Time":28114352,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":37,"Index":37,"Attempt":0,"Launch Time":1546922037865,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":34,"Index":34,"Attempt":0,"Launch Time":1546922037822,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037866,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":24203,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":39843327,"Value":2701731669,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":39,"Value":2865,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":979795,"Value":2448064955,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6550,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":979795,"Executor Run Time":39,"Executor CPU Time":39843327,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":38,"Index":38,"Attempt":0,"Launch Time":1546922037890,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":36,"Index":36,"Attempt":0,"Launch Time":1546922037857,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037890,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":24994,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":27673779,"Value":2729405448,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":27,"Value":2892,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1462274,"Value":2449527229,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6552,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1462274,"Executor Run Time":27,"Executor CPU Time":27673779,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":39,"Index":39,"Attempt":0,"Launch Time":1546922037894,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":19,"Attempt":0,"Launch Time":1546922037639,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037894,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":25785,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":250723087,"Value":2980128535,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":250,"Value":3142,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1520638,"Value":2451047867,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6554,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1520638,"Executor Run Time":250,"Executor CPU Time":250723087,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":40,"Index":40,"Attempt":0,"Launch Time":1546922037917,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":37,"Index":37,"Attempt":0,"Launch Time":1546922037865,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037917,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1522,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":26619,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":40203057,"Value":3020331592,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":46,"Value":3188,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":987313,"Value":2452035180,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6556,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":987313,"Executor Run Time":46,"Executor CPU Time":40203057,"Result Size":834,"JVM GC Time":6,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":41,"Index":41,"Attempt":0,"Launch Time":1546922037919,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":23,"Attempt":0,"Launch Time":1546922037665,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037919,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1528,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":27453,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":241618343,"Value":3261949935,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":248,"Value":3436,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1531323,"Value":2453566503,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6558,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1531323,"Executor Run Time":248,"Executor CPU Time":241618343,"Result Size":834,"JVM GC Time":6,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":42,"Index":42,"Attempt":0,"Launch Time":1546922037922,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":38,"Index":38,"Attempt":0,"Launch Time":1546922037890,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037923,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":28201,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":28219664,"Value":3290169599,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":29,"Value":3465,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1047569,"Value":2454614072,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":1047569,"Executor Run Time":29,"Executor CPU Time":28219664,"Result Size":748,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":43,"Index":43,"Attempt":0,"Launch Time":1546922037949,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":24,"Attempt":0,"Launch Time":1546922037671,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037952,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1534,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":29035,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":239368713,"Value":3529538312,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":246,"Value":3711,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1606167,"Value":2456220239,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":6561,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":1606167,"Executor Run Time":246,"Executor CPU Time":239368713,"Result Size":834,"JVM GC Time":6,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":44,"Index":44,"Attempt":0,"Launch Time":1546922037952,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":20,"Attempt":0,"Launch Time":1546922037641,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037953,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":29826,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":281427189,"Value":3810965501,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":282,"Value":3993,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1654852,"Value":2457875091,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6563,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1654852,"Executor Run Time":282,"Executor CPU Time":281427189,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":45,"Index":45,"Attempt":0,"Launch Time":1546922037953,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":29,"Attempt":0,"Launch Time":1546922037744,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037953,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":30617,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":188231506,"Value":3999197007,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":188,"Value":4181,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1169748,"Value":2459044839,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6565,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1169748,"Executor Run Time":188,"Executor CPU Time":188231506,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":46,"Index":46,"Attempt":0,"Launch Time":1546922037954,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":39,"Index":39,"Attempt":0,"Launch Time":1546922037894,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037954,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":31408,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":42202483,"Value":4041399490,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":42,"Value":4223,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1131763,"Value":2460176602,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6567,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1131763,"Executor Run Time":42,"Executor CPU Time":42202483,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":47,"Index":47,"Attempt":0,"Launch Time":1546922037955,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":41,"Index":41,"Attempt":0,"Launch Time":1546922037919,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037955,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":32242,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":32046930,"Value":4073446420,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":31,"Value":4254,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1193077,"Value":2461369679,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6569,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1193077,"Executor Run Time":31,"Executor CPU Time":32046930,"Result Size":834,"JVM GC Time":0,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":48,"Index":48,"Attempt":0,"Launch Time":1546922037990,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":47,"Index":47,"Attempt":0,"Launch Time":1546922037955,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037990,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":33033,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":31475569,"Value":4104921989,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":31,"Value":4285,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1060814,"Value":2462430493,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6571,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1060814,"Executor Run Time":31,"Executor CPU Time":31475569,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":49,"Index":49,"Attempt":0,"Launch Time":1546922037991,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":46,"Index":46,"Attempt":0,"Launch Time":1546922037954,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037992,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":33781,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":33893704,"Value":4138815693,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":35,"Value":4320,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":995078,"Value":2463425571,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":995078,"Executor Run Time":35,"Executor CPU Time":33893704,"Result Size":748,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":50,"Index":50,"Attempt":0,"Launch Time":1546922038023,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":49,"Index":49,"Attempt":0,"Launch Time":1546922037991,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038024,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":34529,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":27659093,"Value":4166474786,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":29,"Value":4349,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":936644,"Value":2464362215,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":936644,"Executor Run Time":29,"Executor CPU Time":27659093,"Result Size":748,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":51,"Index":51,"Attempt":0,"Launch Time":1546922038025,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":48,"Index":48,"Attempt":0,"Launch Time":1546922037990,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038025,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":35320,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":30657164,"Value":4197131950,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":30,"Value":4379,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1037555,"Value":2465399770,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6572,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1037555,"Executor Run Time":30,"Executor CPU Time":30657164,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":52,"Index":52,"Attempt":0,"Launch Time":1546922038060,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":51,"Index":51,"Attempt":0,"Launch Time":1546922038025,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038061,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":36111,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":31510921,"Value":4228642871,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":32,"Value":4411,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":895198,"Value":2466294968,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6573,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":895198,"Executor Run Time":32,"Executor CPU Time":31510921,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":53,"Index":53,"Attempt":0,"Launch Time":1546922038063,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":50,"Index":50,"Attempt":0,"Launch Time":1546922038023,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038064,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1540,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":877,"Value":36988,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":30458180,"Value":4259101051,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":35,"Value":4446,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":955927,"Value":2467250895,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6575,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":955927,"Executor Run Time":35,"Executor CPU Time":30458180,"Result Size":877,"JVM GC Time":6,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":54,"Index":54,"Attempt":0,"Launch Time":1546922038095,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":52,"Index":52,"Attempt":0,"Launch Time":1546922038060,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038095,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":37779,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":31390887,"Value":4290491938,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":31,"Value":4477,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":893866,"Value":2468144761,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6576,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":893866,"Executor Run Time":31,"Executor CPU Time":31390887,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":55,"Index":55,"Attempt":0,"Launch Time":1546922038097,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":53,"Index":53,"Attempt":0,"Launch Time":1546922038063,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038097,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":38570,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":28537105,"Value":4319029043,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":28,"Value":4505,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1137737,"Value":2469282498,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6578,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1137737,"Executor Run Time":28,"Executor CPU Time":28537105,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":56,"Index":56,"Attempt":0,"Launch Time":1546922038113,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":31,"Index":31,"Attempt":0,"Launch Time":1546922037779,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038113,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1546,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":39404,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":320279036,"Value":4639308079,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":327,"Value":4832,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2309726,"Value":2471592224,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6580,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":2309726,"Executor Run Time":327,"Executor CPU Time":320279036,"Result Size":834,"JVM GC Time":6,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":57,"Index":57,"Attempt":0,"Launch Time":1546922038131,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":54,"Index":54,"Attempt":0,"Launch Time":1546922038095,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038131,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":40195,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":32175125,"Value":4671483204,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":32,"Value":4864,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":861437,"Value":2472453661,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6581,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":861437,"Executor Run Time":32,"Executor CPU Time":32175125,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":58,"Index":58,"Attempt":0,"Launch Time":1546922038137,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":44,"Index":44,"Attempt":0,"Launch Time":1546922037952,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038137,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1552,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":41029,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":175258771,"Value":4846741975,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":180,"Value":5044,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1272588,"Value":2473726249,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6583,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1272588,"Executor Run Time":180,"Executor CPU Time":175258771,"Result Size":834,"JVM GC Time":6,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":59,"Index":59,"Attempt":0,"Launch Time":1546922038148,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":55,"Index":55,"Attempt":0,"Launch Time":1546922038097,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038148,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":41820,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":46067636,"Value":4892809611,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":47,"Value":5091,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1539643,"Value":2475265892,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6584,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1539643,"Executor Run Time":47,"Executor CPU Time":46067636,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":60,"Index":60,"Attempt":0,"Launch Time":1546922038168,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":57,"Index":57,"Attempt":0,"Launch Time":1546922038131,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038169,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":42611,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":33746415,"Value":4926556026,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":34,"Value":5125,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":939846,"Value":2476205738,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6585,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":939846,"Executor Run Time":34,"Executor CPU Time":33746415,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":61,"Index":61,"Attempt":0,"Launch Time":1546922038175,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":40,"Index":40,"Attempt":0,"Launch Time":1546922037917,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038175,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":43402,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":253536114,"Value":5180092140,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":254,"Value":5379,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1121119,"Value":2477326857,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6586,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1121119,"Executor Run Time":254,"Executor CPU Time":253536114,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":62,"Index":62,"Attempt":0,"Launch Time":1546922038185,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":59,"Index":59,"Attempt":0,"Launch Time":1546922038148,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038185,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":44193,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":33229424,"Value":5213321564,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":33,"Value":5412,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":856494,"Value":2478183351,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6587,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":856494,"Executor Run Time":33,"Executor CPU Time":33229424,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":63,"Index":63,"Attempt":0,"Launch Time":1546922038204,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":60,"Index":60,"Attempt":0,"Launch Time":1546922038168,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038204,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":44941,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":31762584,"Value":5245084148,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":32,"Value":5444,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":819419,"Value":2479002770,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":819419,"Executor Run Time":32,"Executor CPU Time":31762584,"Result Size":748,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":64,"Index":64,"Attempt":0,"Launch Time":1546922038217,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":42,"Index":42,"Attempt":0,"Launch Time":1546922037922,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038217,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1558,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":45775,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":254396405,"Value":5499480553,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":260,"Value":5704,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1523395,"Value":2480526165,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6589,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1523395,"Executor Run Time":260,"Executor CPU Time":254396405,"Result Size":834,"JVM GC Time":6,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":65,"Index":65,"Attempt":0,"Launch Time":1546922038218,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":62,"Index":62,"Attempt":0,"Launch Time":1546922038185,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038218,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":46523,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":29041950,"Value":5528522503,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":30,"Value":5734,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":920529,"Value":2481446694,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":920529,"Executor Run Time":30,"Executor CPU Time":29041950,"Result Size":748,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":66,"Index":66,"Attempt":0,"Launch Time":1546922038220,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":45,"Index":45,"Attempt":0,"Launch Time":1546922037953,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038221,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1564,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":47357,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":257839847,"Value":5786362350,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":264,"Value":5998,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1031380,"Value":2482478074,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6590,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1031380,"Executor Run Time":264,"Executor CPU Time":257839847,"Result Size":834,"JVM GC Time":6,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":67,"Index":67,"Attempt":0,"Launch Time":1546922038239,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":63,"Index":63,"Attempt":0,"Launch Time":1546922038204,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038239,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":4,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":48191,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":31072580,"Value":5817434930,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":30,"Value":6028,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":960275,"Value":2483438349,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6592,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":960275,"Executor Run Time":30,"Executor CPU Time":31072580,"Result Size":834,"JVM GC Time":0,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":68,"Index":68,"Attempt":0,"Launch Time":1546922038251,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":64,"Index":64,"Attempt":0,"Launch Time":1546922038217,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038252,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":5,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":48982,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":30559817,"Value":5847994747,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":31,"Value":6059,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":860099,"Value":2484298448,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":860099,"Executor Run Time":31,"Executor CPU Time":30559817,"Result Size":791,"JVM GC Time":0,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":69,"Index":69,"Attempt":0,"Launch Time":1546922038253,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":43,"Index":43,"Attempt":0,"Launch Time":1546922037949,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038253,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":49773,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":296727303,"Value":6144722050,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":297,"Value":6356,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1526695,"Value":2485825143,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6593,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1526695,"Executor Run Time":297,"Executor CPU Time":296727303,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":70,"Index":70,"Attempt":0,"Launch Time":1546922038275,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":67,"Index":67,"Attempt":0,"Launch Time":1546922038239,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038276,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":50521,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":33148590,"Value":6177870640,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":34,"Value":6390,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":841498,"Value":2486666641,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":841498,"Executor Run Time":34,"Executor CPU Time":33148590,"Result Size":748,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":71,"Index":71,"Attempt":0,"Launch Time":1546922038291,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":68,"Index":68,"Attempt":0,"Launch Time":1546922038251,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038292,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":51269,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":35277840,"Value":6213148480,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":36,"Value":6426,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":981161,"Value":2487647802,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":981161,"Executor Run Time":36,"Executor CPU Time":35277840,"Result Size":748,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":72,"Index":72,"Attempt":0,"Launch Time":1546922038294,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":58,"Index":58,"Attempt":0,"Launch Time":1546922038137,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038295,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":52060,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":152692713,"Value":6365841193,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":153,"Value":6579,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1007896,"Value":2488655698,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6594,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1007896,"Executor Run Time":153,"Executor CPU Time":152692713,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":73,"Index":73,"Attempt":0,"Launch Time":1546922038313,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":70,"Index":70,"Attempt":0,"Launch Time":1546922038275,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038313,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":52808,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":33306826,"Value":6399148019,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":34,"Value":6613,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":956832,"Value":2489612530,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":956832,"Executor Run Time":34,"Executor CPU Time":33306826,"Result Size":748,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":74,"Index":74,"Attempt":0,"Launch Time":1546922038329,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":71,"Index":71,"Attempt":0,"Launch Time":1546922038291,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038330,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":53599,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":33756595,"Value":6432904614,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":33,"Value":6646,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":882246,"Value":2490494776,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6595,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":882246,"Executor Run Time":33,"Executor CPU Time":33756595,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":75,"Index":75,"Attempt":0,"Launch Time":1546922038347,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":73,"Index":73,"Attempt":0,"Launch Time":1546922038313,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038348,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":54390,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":30684763,"Value":6463589377,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":30,"Value":6676,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":943134,"Value":2491437910,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6597,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":943134,"Executor Run Time":30,"Executor CPU Time":30684763,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":76,"Index":76,"Attempt":0,"Launch Time":1546922038368,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":74,"Index":74,"Attempt":0,"Launch Time":1546922038329,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038368,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":55181,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":34465927,"Value":6498055304,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":34,"Value":6710,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":964913,"Value":2492402823,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6598,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":964913,"Executor Run Time":34,"Executor CPU Time":34465927,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":77,"Index":77,"Attempt":0,"Launch Time":1546922038390,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":75,"Index":75,"Attempt":0,"Launch Time":1546922038347,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038391,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1570,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":56015,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":31793636,"Value":6529848940,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":37,"Value":6747,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":975320,"Value":2493378143,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6600,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":975320,"Executor Run Time":37,"Executor CPU Time":31793636,"Result Size":834,"JVM GC Time":6,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":78,"Index":78,"Attempt":0,"Launch Time":1546922038409,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":76,"Index":76,"Attempt":0,"Launch Time":1546922038368,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038409,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":56806,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":36274008,"Value":6566122948,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":36,"Value":6783,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":972306,"Value":2494350449,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6602,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":972306,"Executor Run Time":36,"Executor CPU Time":36274008,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":79,"Index":79,"Attempt":0,"Launch Time":1546922038416,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":61,"Index":61,"Attempt":0,"Launch Time":1546922038175,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038416,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1576,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":57640,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":230475162,"Value":6796598110,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":236,"Value":7019,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":989658,"Value":2495340107,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6603,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":989658,"Executor Run Time":236,"Executor CPU Time":230475162,"Result Size":834,"JVM GC Time":6,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":80,"Index":80,"Attempt":0,"Launch Time":1546922038424,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":56,"Index":56,"Attempt":0,"Launch Time":1546922038113,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038424,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1582,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":58474,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":301424170,"Value":7098022280,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":308,"Value":7327,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1012899,"Value":2496353006,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6604,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1012899,"Executor Run Time":308,"Executor CPU Time":301424170,"Result Size":834,"JVM GC Time":6,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":81,"Index":81,"Attempt":0,"Launch Time":1546922038436,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":69,"Index":69,"Attempt":0,"Launch Time":1546922038253,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038436,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1588,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":59308,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":172594019,"Value":7270616299,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":179,"Value":7506,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1197294,"Value":2497550300,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6605,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1197294,"Executor Run Time":179,"Executor CPU Time":172594019,"Result Size":834,"JVM GC Time":6,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":82,"Index":82,"Attempt":0,"Launch Time":1546922038448,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":78,"Index":78,"Attempt":0,"Launch Time":1546922038409,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038448,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":60099,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":34408818,"Value":7305025117,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":34,"Value":7540,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":989551,"Value":2498539851,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6606,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":989551,"Executor Run Time":34,"Executor CPU Time":34408818,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":83,"Index":83,"Attempt":0,"Launch Time":1546922038483,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":65,"Index":65,"Attempt":0,"Launch Time":1546922038218,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038483,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1594,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":60890,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":254383686,"Value":7559408803,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":261,"Value":7801,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1135212,"Value":2499675063,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":1135212,"Executor Run Time":261,"Executor CPU Time":254383686,"Result Size":791,"JVM GC Time":6,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":84,"Index":84,"Attempt":0,"Launch Time":1546922038492,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":82,"Index":82,"Attempt":0,"Launch Time":1546922038448,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038493,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1600,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":61681,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":34883743,"Value":7594292546,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":42,"Value":7843,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1012178,"Value":2500687241,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":1012178,"Executor Run Time":42,"Executor CPU Time":34883743,"Result Size":791,"JVM GC Time":6,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":85,"Index":85,"Attempt":0,"Launch Time":1546922038495,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":66,"Index":66,"Attempt":0,"Launch Time":1546922038220,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038495,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1606,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":62515,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":264938303,"Value":7859230849,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":270,"Value":8113,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":934827,"Value":2501622068,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6608,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":934827,"Executor Run Time":270,"Executor CPU Time":264938303,"Result Size":834,"JVM GC Time":6,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":86,"Index":86,"Attempt":0,"Launch Time":1546922038507,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":77,"Index":77,"Attempt":0,"Launch Time":1546922038390,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038507,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":63263,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":112064631,"Value":7971295480,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":114,"Value":8227,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1053290,"Value":2502675358,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":1053290,"Executor Run Time":114,"Executor CPU Time":112064631,"Result Size":748,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":87,"Index":87,"Attempt":0,"Launch Time":1546922038517,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":80,"Index":80,"Attempt":0,"Launch Time":1546922038424,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038518,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":64054,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":89355066,"Value":8060650546,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":89,"Value":8316,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":998260,"Value":2503673618,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6610,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":998260,"Executor Run Time":89,"Executor CPU Time":89355066,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":88,"Index":88,"Attempt":0,"Launch Time":1546922038533,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":84,"Index":84,"Attempt":0,"Launch Time":1546922038492,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038533,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":64802,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":35725959,"Value":8096376505,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":37,"Value":8353,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1128802,"Value":2504802420,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":1128802,"Executor Run Time":37,"Executor CPU Time":35725959,"Result Size":748,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":89,"Index":89,"Attempt":0,"Launch Time":1546922038536,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":81,"Index":81,"Attempt":0,"Launch Time":1546922038436,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038537,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":65593,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":96824193,"Value":8193200698,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":96,"Value":8449,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":857593,"Value":2505660013,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6612,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":857593,"Executor Run Time":96,"Executor CPU Time":96824193,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":90,"Index":90,"Attempt":0,"Launch Time":1546922038550,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":72,"Index":72,"Attempt":0,"Launch Time":1546922038294,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038550,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1612,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":66427,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":245593808,"Value":8438794506,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":251,"Value":8700,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1199629,"Value":2506859642,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6614,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1199629,"Executor Run Time":251,"Executor CPU Time":245593808,"Result Size":834,"JVM GC Time":6,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":91,"Index":91,"Attempt":0,"Launch Time":1546922038551,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":79,"Index":79,"Attempt":0,"Launch Time":1546922038416,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038552,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":67175,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":130530302,"Value":8569324808,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":132,"Value":8832,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1409511,"Value":2508269153,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":1409511,"Executor Run Time":132,"Executor CPU Time":130530302,"Result Size":748,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":92,"Index":92,"Attempt":0,"Launch Time":1546922038574,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":88,"Index":88,"Attempt":0,"Launch Time":1546922038533,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038574,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":67966,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":36971815,"Value":8606296623,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":36,"Value":8868,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1067084,"Value":2509336237,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6616,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1067084,"Executor Run Time":36,"Executor CPU Time":36971815,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":93,"Index":93,"Attempt":0,"Launch Time":1546922038612,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":86,"Index":86,"Attempt":0,"Launch Time":1546922038507,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038612,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":68757,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":100349075,"Value":8706645698,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":100,"Value":8968,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1073198,"Value":2510409435,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6617,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1073198,"Executor Run Time":100,"Executor CPU Time":100349075,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":94,"Index":94,"Attempt":0,"Launch Time":1546922038614,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":85,"Index":85,"Attempt":0,"Launch Time":1546922038495,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038614,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":6,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":69548,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":113541356,"Value":8820187054,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":114,"Value":9082,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":904680,"Value":2511314115,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":904680,"Executor Run Time":114,"Executor CPU Time":113541356,"Result Size":791,"JVM GC Time":0,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":95,"Index":95,"Attempt":0,"Launch Time":1546922038621,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":87,"Index":87,"Attempt":0,"Launch Time":1546922038517,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038621,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":70339,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":100446517,"Value":8920633571,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":100,"Value":9182,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":895108,"Value":2512209223,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6618,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":895108,"Executor Run Time":100,"Executor CPU Time":100446517,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":96,"Index":96,"Attempt":0,"Launch Time":1546922038636,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":89,"Index":89,"Attempt":0,"Launch Time":1546922038536,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038637,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":7,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":71130,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":96681743,"Value":9017315314,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":97,"Value":9279,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":881317,"Value":2513090540,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":881317,"Executor Run Time":97,"Executor CPU Time":96681743,"Result Size":791,"JVM GC Time":0,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":97,"Index":97,"Attempt":0,"Launch Time":1546922038640,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":92,"Index":92,"Attempt":0,"Launch Time":1546922038574,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038641,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":71921,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":61426798,"Value":9078742112,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":61,"Value":9340,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1009168,"Value":2514099708,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6620,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1009168,"Executor Run Time":61,"Executor CPU Time":61426798,"Result Size":791,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":98,"Index":98,"Attempt":0,"Launch Time":1546922038664,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":91,"Index":91,"Attempt":0,"Launch Time":1546922038551,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038664,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":72669,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":108784588,"Value":9187526700,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":110,"Value":9450,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1001879,"Value":2515101587,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":1001879,"Executor Run Time":110,"Executor CPU Time":108784588,"Result Size":748,"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":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":99,"Index":99,"Attempt":0,"Launch Time":1546922038680,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":94,"Index":94,"Attempt":0,"Launch Time":1546922038614,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038681,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":73460,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":60115853,"Value":9247642553,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":61,"Value":9511,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1466213,"Value":2516567800,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6621,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1466213,"Executor Run Time":61,"Executor CPU Time":60115853,"Result Size":791,"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":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":83,"Index":83,"Attempt":0,"Launch Time":1546922038483,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038705,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":74251,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":214285427,"Value":9461927980,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":214,"Value":9725,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1060594,"Value":2517628394,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6623,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1060594,"Executor Run Time":214,"Executor CPU Time":214285427,"Result Size":791,"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":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":97,"Index":97,"Attempt":0,"Launch Time":1546922038640,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038705,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":75042,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":57994238,"Value":9519922218,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":58,"Value":9783,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1405974,"Value":2519034368,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6625,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1405974,"Executor Run Time":58,"Executor CPU Time":57994238,"Result Size":791,"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":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":93,"Index":93,"Attempt":0,"Launch Time":1546922038612,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038717,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":75833,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":101684514,"Value":9621606732,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":101,"Value":9884,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":968858,"Value":2520003226,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6627,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":968858,"Executor Run Time":101,"Executor CPU Time":101684514,"Result Size":791,"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":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":99,"Index":99,"Attempt":0,"Launch Time":1546922038680,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038724,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":76624,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":38985492,"Value":9660592224,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":39,"Value":9923,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1411378,"Value":2521414604,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6629,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1411378,"Executor Run Time":39,"Executor CPU Time":38985492,"Result Size":791,"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":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":90,"Index":90,"Attempt":0,"Launch Time":1546922038550,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038725,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":77415,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":169783721,"Value":9830375945,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":170,"Value":10093,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":805689,"Value":2522220293,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6630,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":805689,"Executor Run Time":170,"Executor CPU Time":169783721,"Result Size":791,"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":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":95,"Index":95,"Attempt":0,"Launch Time":1546922038621,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038726,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":78206,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":101866107,"Value":9932242052,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":102,"Value":10195,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":805277,"Value":2523025570,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6631,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":805277,"Executor Run Time":102,"Executor CPU Time":101866107,"Result Size":791,"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":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":96,"Index":96,"Attempt":0,"Launch Time":1546922038636,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038729,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":78954,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":89123011,"Value":10021365063,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":90,"Value":10285,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":896962,"Value":2523922532,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":896962,"Executor Run Time":90,"Executor CPU Time":89123011,"Result Size":748,"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":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":98,"Index":98,"Attempt":0,"Launch Time":1546922038664,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038733,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":79745,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":64532656,"Value":10085897719,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":64,"Value":10349,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1115835,"Value":2525038367,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6633,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1115835,"Executor Run Time":64,"Executor CPU Time":64532656,"Result Size":791,"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":"driver","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":211171816,"JVMOffHeapMemory":90237256,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":4876,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":4876,"OffHeapUnifiedMemory":0,"DirectPoolMemory":806275,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":2646888448,"ProcessTreeJVMRSSMemory":520900608,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":8,"MinorGCTime":374,"MajorGCCount":3,"MajorGCTime":170}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"reduce at SparkPi.scala:38","Number of Tasks":100,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"map\"}","Callsite":"map at SparkPi.scala:34","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":100,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at SparkPi.scala:34","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":100,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.reduce(RDD.scala:1031)\norg.apache.spark.examples.SparkPi$.main(SparkPi.scala:38)\norg.apache.spark.examples.SparkPi.main(SparkPi.scala)\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)\norg.apache.spark.deploy.yarn.ApplicationMaster$$anon$2.run(ApplicationMaster.scala:677)","Submission Time":1546922036383,"Completion Time":1546922038734,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Value":10349,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Value":1612,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Value":79745,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Value":2525038367,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Value":10085897719,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Value":7,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Value":6633,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1546922038738,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerApplicationEnd","Timestamp":1546922038745} diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index 1a071fa77133..6665a890220e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -137,6 +137,8 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers "applications/application_1506645932520_24630151/executors", "executor list with executor process tree metrics json" -> "applications/application_1538416563558_0014/executors", + "executor list with executor garbage collection metrics json" -> + "applications/application_1536831636016_59384/1/executors", "stage list json" -> "applications/local-1422981780767/stages", "complete stage list json" -> "applications/local-1422981780767/stages?status=complete", "failed stage list json" -> "applications/local-1422981780767/stages?status=failed", diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index 40521f073217..350fc2a48f17 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -285,66 +285,66 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit // with different peak updates for each executor createExecutorMetricsUpdateEvent(1, new ExecutorMetrics(Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L, 7500L, 3500L, - 6500L, 2500L, 5500L, 1500L))), + 6500L, 2500L, 5500L, 1500L, 10L, 90L, 2L, 20L))), createExecutorMetricsUpdateEvent(2, new ExecutorMetrics(Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L, 8500L, 3500L, - 7500L, 2500L, 6500L, 1500L))), + 7500L, 2500L, 6500L, 1500L, 10L, 90L, 2L, 20L))), // exec 1: new stage 0 peaks for metrics at indexes: 2, 4, 6 createExecutorMetricsUpdateEvent(1, new ExecutorMetrics(Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L, 8000L, 4000L, - 7000L, 3000L, 6000L, 2000L))), + 7000L, 3000L, 6000L, 2000L, 10L, 90L, 2L, 20L))), // exec 2: new stage 0 peaks for metrics at indexes: 0, 4, 6 createExecutorMetricsUpdateEvent(2, new ExecutorMetrics(Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L, 9000L, 4000L, - 8000L, 3000L, 7000L, 2000L))), + 8000L, 3000L, 7000L, 2000L, 10L, 90L, 2L, 20L))), // exec 1: new stage 0 peaks for metrics at indexes: 5, 7 createExecutorMetricsUpdateEvent(1, new ExecutorMetrics(Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L, 8000L, 3500L, - 7000L, 2500L, 6000L, 1500L))), + 7000L, 2500L, 6000L, 1500L, 10L, 90L, 2L, 20L))), // exec 2: new stage 0 peaks for metrics at indexes: 0, 5, 6, 7, 8 createExecutorMetricsUpdateEvent(2, new ExecutorMetrics(Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L, 8500L, 3500L, - 7500L, 2500L, 6500L, 1500L))), + 7500L, 2500L, 6500L, 1500L, 10L, 90L, 2L, 20L))), // now start stage 1, one more metric update for each executor, and new // peaks for some stage 1 metrics (as listed), initialize stage 1 peaks createStageSubmittedEvent(1), // exec 1: new stage 0 peaks for metrics at indexes: 0, 3, 7; initialize stage 1 peaks createExecutorMetricsUpdateEvent(1, new ExecutorMetrics(Array(5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, - 0L, 5000L, 3000L, 4000L, 2000L, 3000L, 1000L))), + 0L, 5000L, 3000L, 4000L, 2000L, 3000L, 1000L, 10L, 90L, 2L, 20L))), // exec 2: new stage 0 peaks for metrics at indexes: 0, 1, 3, 6, 7, 9; // initialize stage 1 peaks createExecutorMetricsUpdateEvent(2, new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, - 40L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L))), + 40L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L, 10L, 90L, 2L, 20L))), // complete stage 0, and 3 more updates for each executor with just // stage 1 running createStageCompletedEvent(0), // exec 1: new stage 1 peaks for metrics at indexes: 0, 1, 3 createExecutorMetricsUpdateEvent(1, new ExecutorMetrics(Array(6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L, 5000L, 3000L, - 4000L, 2000L, 3000L, 1000L))), + 4000L, 2000L, 3000L, 1000L, 10L, 90L, 2L, 20L))), // exec 2: new stage 1 peaks for metrics at indexes: 3, 4, 7, 8 createExecutorMetricsUpdateEvent(2, new ExecutorMetrics(Array(5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, - 20L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L, 5000L, 2000L))), + 20L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L, 10L, 90L, 2L, 20L))), // exec 1: new stage 1 peaks for metrics at indexes: 0, 4, 5, 7 createExecutorMetricsUpdateEvent(1, new ExecutorMetrics(Array(7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L, 3000L, 2500L, - 2000L, 1500L, 1000L, 500L))), + 2000L, 1500L, 1000L, 500L, 10L, 90L, 2L, 20L))), // exec 2: new stage 1 peak for metrics at index: 7 createExecutorMetricsUpdateEvent(2, new ExecutorMetrics(Array(5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, - 20L, 7000L, 3000L, 6000L, 2000L, 5000L, 1000L))), + 20L, 7000L, 3000L, 6000L, 2000L, 5000L, 1000L, 10L, 90L, 2L, 20L))), // exec 1: no new stage 1 peaks createExecutorMetricsUpdateEvent(1, new ExecutorMetrics(Array(5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, - 0L, 4000L, 2500L, 3000L, 1500L, 2000L, 500L))), + 0L, 4000L, 2500L, 3000L, 1500L, 2000L, 500L, 10L, 90L, 2L, 20L))), createExecutorRemovedEvent(1), // exec 2: new stage 1 peak for metrics at index: 6 createExecutorMetricsUpdateEvent(2, new ExecutorMetrics(Array(4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L, 7000L, - 4000L, 6000L, 3000L, 5000L, 2000L))), + 4000L, 6000L, 3000L, 5000L, 2000L, 10L, 90L, 2L, 20L))), createStageCompletedEvent(1), SparkListenerApplicationEnd(1000L)) @@ -362,19 +362,19 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit ((0, "1"), new SparkListenerStageExecutorMetrics("1", 0, 0, new ExecutorMetrics(Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, - 70L, 20L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L)))), + 70L, 20L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L, 10L, 90L, 2L, 20L)))), ((0, "2"), new SparkListenerStageExecutorMetrics("2", 0, 0, new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, - 80L, 40L, 9000L, 4000L, 8000L, 3000L, 7000L, 2000L)))), + 80L, 40L, 9000L, 4000L, 8000L, 3000L, 7000L, 2000L, 10L, 90L, 2L, 20L)))), ((1, "1"), new SparkListenerStageExecutorMetrics("1", 1, 0, new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, - 50L, 0L, 5000L, 3000L, 4000L, 2000L, 3000L, 1000L)))), + 50L, 0L, 5000L, 3000L, 4000L, 2000L, 3000L, 1000L, 10L, 90L, 2L, 20L)))), ((1, "2"), new SparkListenerStageExecutorMetrics("2", 1, 0, new ExecutorMetrics(Array(7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, - 40L, 40L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L))))) + 40L, 40L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L, 10L, 90L, 2L, 20L))))) // Verify the log file contains the expected events. // Posted events should be logged, except for ExecutorMetricsUpdate events -- these // are consolidated, and the peak values for each stage are logged at stage end. diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index c3ff379c84ff..f2f62d6d4df1 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -99,7 +99,7 @@ class JsonProtocolSuite extends SparkFunSuite { .zipWithIndex.map { case (a, i) => a.copy(id = i) } val executorUpdates = new ExecutorMetrics( Array(543L, 123456L, 12345L, 1234L, 123L, 12L, 432L, - 321L, 654L, 765L, 256912L, 123456L, 123456L, 61728L, 30364L, 15182L)) + 321L, 654L, 765L, 256912L, 123456L, 123456L, 61728L, 30364L, 15182L, 10L, 90L, 2L, 20L)) SparkListenerExecutorMetricsUpdate("exec3", Seq((1L, 2, 3, accumUpdates)), Some(executorUpdates)) } @@ -109,7 +109,7 @@ class JsonProtocolSuite extends SparkFunSuite { val stageExecutorMetrics = SparkListenerStageExecutorMetrics("1", 2, 3, new ExecutorMetrics(Array(543L, 123456L, 12345L, 1234L, 123L, 12L, 432L, - 321L, 654L, 765L, 256912L, 123456L, 123456L, 61728L, 30364L, 15182L))) + 321L, 654L, 765L, 256912L, 123456L, 123456L, 61728L, 30364L, 15182L, 10L, 90L, 2L, 20L))) testEvent(stageSubmitted, stageSubmittedJsonString) testEvent(stageCompleted, stageCompletedJsonString) testEvent(taskStart, taskStartJsonString) @@ -888,7 +888,7 @@ private[spark] object JsonProtocolSuite extends Assertions { val executorMetricsUpdate = if (includeExecutorMetrics) { Some(new ExecutorMetrics(Array(123456L, 543L, 0L, 0L, 0L, 0L, 0L, - 0L, 0L, 0L, 256912L, 123456L, 123456L, 61728L, 30364L, 15182L))) + 0L, 0L, 0L, 256912L, 123456L, 123456L, 61728L, 30364L, 15182L, 10L, 90L, 2L, 20L))) } else { None } @@ -2106,7 +2106,11 @@ private[spark] object JsonProtocolSuite extends Assertions { | "ProcessTreePythonVMemory": 123456, | "ProcessTreePythonRSSMemory": 61728, | "ProcessTreeOtherVMemory": 30364, - | "ProcessTreeOtherRSSMemory": 15182 + | "ProcessTreeOtherRSSMemory": 15182, + | "MinorGCCount": 10, + | "MinorGCTime": 90, + | "MajorGCCount": 2, + | "MajorGCTime": 20 | } | |} @@ -2135,7 +2139,11 @@ private[spark] object JsonProtocolSuite extends Assertions { | "ProcessTreePythonVMemory": 123456, | "ProcessTreePythonRSSMemory": 61728, | "ProcessTreeOtherVMemory": 30364, - | "ProcessTreeOtherRSSMemory": 15182 + | "ProcessTreeOtherRSSMemory": 15182, + | "MinorGCCount": 10, + | "MinorGCTime": 90, + | "MajorGCCount": 2, + | "MajorGCTime": 20 | } |} """.stripMargin diff --git a/dev/.rat-excludes b/dev/.rat-excludes index 8239cbc3a381..59e619ba109b 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -83,6 +83,7 @@ app-20161116163331-0000 application_1516285256255_0012 application_1506645932520_24630151 application_1538416563558_0014 +application_1536831636016_59384_1 stat local-1422981759269 local-1422981780767