From 525d99986fe0f1c1f1367d5f3de2b0e448239218 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Sat, 8 Dec 2018 15:32:46 +0900 Subject: [PATCH 01/28] [SPARK-26311][YARN] New feature: custom log URL for stdout/stderr --- docs/running-on-yarn.md | 15 ++++++ .../spark/deploy/yarn/ExecutorRunnable.scala | 49 +++++++++++++++++-- .../org/apache/spark/deploy/yarn/config.scala | 9 ++++ 3 files changed, 70 insertions(+), 3 deletions(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index a7a448fbeb65e..632f059279cbe 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -430,6 +430,21 @@ To use a custom metrics.properties for the application master and executors, upd See spark.yarn.config.gatewayPath. + + spark.yarn.custom.log.url + (none) + + Specifies custom spark log url for supporting external log service rather than NodeManager webapp address. + Spark will support some path variables via patterns. Supported patterns and allocated values are below:

+

+ * `{{HttpScheme}}`: `http`/`https` according to YARN HTTP policy. (Configured via `yarn.http.policy`)

+ * `{{NodeHttpAddress}}`: HTTP URI of the node on Container.

+ * `{{ClusterId}}`: The cluster ID of Resource Manager. (Configured via `yarn.resourcemanager.cluster-id`)

+ * `{{ContainerId}}`: The ID of container.

+ * `{{User}}`: `SPARK_USER` on system environment.

+ * `{{FileName}}`: `stdout`/`stderr`.

+ + spark.yarn.rolledLog.includePattern (none) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 77ce2f65245ae..207be63ed5a48 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -246,13 +246,56 @@ private[yarn] class ExecutorRunnable( sys.env.filterKeys(_.endsWith("USER")).foreach { user => val containerId = ConverterUtils.toString(c.getId) val address = c.getNodeHttpAddress - val baseUrl = s"$httpScheme$address/node/containerlogs/$containerId/$user" - env("SPARK_LOG_URL_STDERR") = s"$baseUrl/stderr?start=-4096" - env("SPARK_LOG_URL_STDOUT") = s"$baseUrl/stdout?start=-4096" + sparkConf.get(config.CUSTOM_LOG_URL) match { + case Some(customUrl) => + val pathVariables = ExecutorRunnable.buildPathVariables(httpScheme, address, + YarnConfiguration.getClusterId(conf), containerId, user) + val envNameToFileNameMap = Map("SPARK_LOG_URL_STDERR" -> "stderr", + "SPARK_LOG_URL_STDOUT" -> "stdout") + val logUrls = ExecutorRunnable.replaceLogUrls(customUrl, pathVariables, + envNameToFileNameMap) + + logUrls.foreach { case (envName, url) => + env(envName) = url + } + case None => + val baseUrl = s"$httpScheme$address/node/containerlogs/$containerId/$user" + env("SPARK_LOG_URL_STDERR") = s"$baseUrl/stderr?start=-4096" + env("SPARK_LOG_URL_STDOUT") = s"$baseUrl/stdout?start=-4096" + } } } env } } + +private[yarn] object ExecutorRunnable { + val LOG_URL_PATTERN_HTTP_SCHEME = "{{HttpScheme}}" + val LOG_URL_PATTERN_NODE_HTTP_ADDRESS = "{{NodeHttpAddress}}" + val LOG_URL_PATTERN_CLUSTER_ID = "{{ClusterId}}" + val LOG_URL_PATTERN_CONTAINER_ID = "{{ContainerId}}" + val LOG_URL_PATTERN_USER = "{{User}}" + val LOG_URL_PATTERN_FILE_NAME = "{{FileName}}" + + def buildPathVariables(httpScheme: String, nodeHttpAddress: String, clusterId: String, + containerId: String, user: String): Map[String, String] = { + Map(LOG_URL_PATTERN_HTTP_SCHEME -> httpScheme, + LOG_URL_PATTERN_NODE_HTTP_ADDRESS -> nodeHttpAddress, + LOG_URL_PATTERN_CLUSTER_ID -> clusterId, + LOG_URL_PATTERN_CONTAINER_ID -> containerId, + LOG_URL_PATTERN_USER -> user) + } + + def replaceLogUrls(logUrlPattern: String, pathVariables: Map[String, String], + envNameToFileNameMap: Map[String, String]): Map[String, String] = { + var replacingUrl = logUrlPattern + pathVariables.foreach { case (pattern, value) => + replacingUrl = replacingUrl.replace(pattern, value) + } + envNameToFileNameMap.map { case (envName, fileName) => + envName -> replacingUrl.replace(LOG_URL_PATTERN_FILE_NAME, fileName) + } + } +} diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala index 16adaec04802e..ff59e70f02153 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala @@ -258,6 +258,15 @@ package object config { "that hosts fs.defaultFS does not need to be listed here.") .fallbackConf(NAMENODES_TO_ACCESS) + /* YARN log url configuration. */ + private[spark] val CUSTOM_LOG_URL = ConfigBuilder("spark.yarn.custom.log.url") + .doc("Specifies custom spark log url for external log service other than node http address. " + + "Spark will support some path variables via patterns. " + + "Supported patterns are 1. `{{HttpScheme}}` 2. `{{NodeHttpAddress}}` 3. `{{ClusterId}}` " + + "4. `{{ContainerId}}` 5. `{{User}}` 6. `{{FileName}}`.") + .stringConf + .createOptional + /* Rolled log aggregation configuration. */ private[spark] val ROLLED_LOG_INCLUDE_PATTERN = From 9b00a11193010eeb3bc5d9b0ef57b87d0b3fe6c9 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Tue, 11 Dec 2018 16:09:43 +0900 Subject: [PATCH 02/28] Address review comments from @vanzin --- docs/running-on-yarn.md | 6 +- .../spark/deploy/yarn/ExecutorRunnable.scala | 78 +++++++++-------- .../org/apache/spark/deploy/yarn/config.scala | 6 +- .../spark/deploy/yarn/YarnLogUrlSuite.scala | 83 +++++++++++++++++++ 4 files changed, 133 insertions(+), 40 deletions(-) create mode 100644 resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnLogUrlSuite.scala diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 632f059279cbe..d2c6af85103f8 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -432,13 +432,13 @@ To use a custom metrics.properties for the application master and executors, upd spark.yarn.custom.log.url - (none) + {{HttpScheme}}{{NodeHttpAddress}}/node/containerlogs/{{ContainerId}}/{{User}}/{{FileName}}?start=-4096 Specifies custom spark log url for supporting external log service rather than NodeManager webapp address. Spark will support some path variables via patterns. Supported patterns and allocated values are below:

- * `{{HttpScheme}}`: `http`/`https` according to YARN HTTP policy. (Configured via `yarn.http.policy`)

- * `{{NodeHttpAddress}}`: HTTP URI of the node on Container.

+ * `{{HttpScheme}}`: `http://`/`https://` according to YARN HTTP policy. (Configured via `yarn.http.policy`)

+ * `{{NodeHttpAddress}}`: The "host:port" of node where container was run.

* `{{ClusterId}}`: The cluster ID of Resource Manager. (Configured via `yarn.resourcemanager.cluster-id`)

* `{{ContainerId}}`: The ID of container.

* `{{User}}`: `SPARK_USER` on system environment.

diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 207be63ed5a48..95afa55d12168 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -24,6 +24,7 @@ import java.util.Collections import scala.collection.JavaConverters._ import scala.collection.mutable.{HashMap, ListBuffer} +import org.apache.hadoop.HadoopIllegalArgumentException import org.apache.hadoop.fs.Path import org.apache.hadoop.io.DataOutputBuffer import org.apache.hadoop.security.UserGroupInformation @@ -57,6 +58,12 @@ private[yarn] class ExecutorRunnable( var rpc: YarnRPC = YarnRPC.create(conf) var nmClient: NMClient = _ + val clusterId: Option[String] = try { + Some(YarnConfiguration.getClusterId(conf)) + } catch { + case _: HadoopIllegalArgumentException => None + } + def run(): Unit = { logDebug("Starting Executor Container") nmClient = NMClient.createNMClient() @@ -247,23 +254,15 @@ private[yarn] class ExecutorRunnable( val containerId = ConverterUtils.toString(c.getId) val address = c.getNodeHttpAddress - sparkConf.get(config.CUSTOM_LOG_URL) match { - case Some(customUrl) => - val pathVariables = ExecutorRunnable.buildPathVariables(httpScheme, address, - YarnConfiguration.getClusterId(conf), containerId, user) - val envNameToFileNameMap = Map("SPARK_LOG_URL_STDERR" -> "stderr", - "SPARK_LOG_URL_STDOUT" -> "stdout") - val logUrls = ExecutorRunnable.replaceLogUrls(customUrl, pathVariables, - envNameToFileNameMap) - - logUrls.foreach { case (envName, url) => - env(envName) = url - } - case None => - val baseUrl = s"$httpScheme$address/node/containerlogs/$containerId/$user" - env("SPARK_LOG_URL_STDERR") = s"$baseUrl/stderr?start=-4096" - env("SPARK_LOG_URL_STDOUT") = s"$baseUrl/stdout?start=-4096" - } + val customLogUrl = sparkConf.get(config.CUSTOM_LOG_URL) + + val envNameToFileNameMap = Map("SPARK_LOG_URL_STDERR" -> "stderr", + "SPARK_LOG_URL_STDOUT" -> "stdout") + val logUrls = ExecutorRunnable.buildLogUrls(customLogUrl, httpScheme, address, + clusterId, containerId, user, envNameToFileNameMap) + logUrls.foreach { case (envName, url) => + env(envName) = url + } } } @@ -272,30 +271,39 @@ private[yarn] class ExecutorRunnable( } private[yarn] object ExecutorRunnable { - val LOG_URL_PATTERN_HTTP_SCHEME = "{{HttpScheme}}" - val LOG_URL_PATTERN_NODE_HTTP_ADDRESS = "{{NodeHttpAddress}}" - val LOG_URL_PATTERN_CLUSTER_ID = "{{ClusterId}}" - val LOG_URL_PATTERN_CONTAINER_ID = "{{ContainerId}}" - val LOG_URL_PATTERN_USER = "{{User}}" - val LOG_URL_PATTERN_FILE_NAME = "{{FileName}}" - - def buildPathVariables(httpScheme: String, nodeHttpAddress: String, clusterId: String, - containerId: String, user: String): Map[String, String] = { - Map(LOG_URL_PATTERN_HTTP_SCHEME -> httpScheme, - LOG_URL_PATTERN_NODE_HTTP_ADDRESS -> nodeHttpAddress, - LOG_URL_PATTERN_CLUSTER_ID -> clusterId, - LOG_URL_PATTERN_CONTAINER_ID -> containerId, - LOG_URL_PATTERN_USER -> user) - } + def buildLogUrls( + logUrlPattern: String, + httpScheme: String, + nodeHttpAddress: String, + clusterId: Option[String], + containerId: String, + user: String, + envNameToFileNameMap: Map[String, String]): Map[String, String] = { + val optionalPathVariable: Map[String, Option[String]] = Map("{{ClusterId}}" -> clusterId) + val pathVariables: Map[String, String] = Map("{{HttpScheme}}" -> httpScheme, + "{{NodeHttpAddress}}" -> nodeHttpAddress, + "{{ContainerId}}" -> containerId, + "{{User}}" -> user) - def replaceLogUrls(logUrlPattern: String, pathVariables: Map[String, String], - envNameToFileNameMap: Map[String, String]): Map[String, String] = { var replacingUrl = logUrlPattern + + optionalPathVariable.foreach { + case (pattern, Some(value)) if replacingUrl.contains(pattern) => + replacingUrl = replacingUrl.replace(pattern, value) + + case (pattern, None) if replacingUrl.contains(pattern) => + throw new IllegalArgumentException(s"Pattern $pattern is specified in " + + s"log URL, but Spark can't retrieve the value. Please check relevant YARN config.") + + case _ => + } + pathVariables.foreach { case (pattern, value) => replacingUrl = replacingUrl.replace(pattern, value) } + envNameToFileNameMap.map { case (envName, fileName) => - envName -> replacingUrl.replace(LOG_URL_PATTERN_FILE_NAME, fileName) + envName -> replacingUrl.replace("{{FileName}}", fileName) } } } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala index ff59e70f02153..7214b09f772b0 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala @@ -263,9 +263,11 @@ package object config { .doc("Specifies custom spark log url for external log service other than node http address. " + "Spark will support some path variables via patterns. " + "Supported patterns are 1. `{{HttpScheme}}` 2. `{{NodeHttpAddress}}` 3. `{{ClusterId}}` " + - "4. `{{ContainerId}}` 5. `{{User}}` 6. `{{FileName}}`.") + "4. `{{ContainerId}}` 5. `{{User}}` 6. `{{FileName}}`. The default value is pointing to " + + "NodeManager's container log URL.") .stringConf - .createOptional + .createWithDefault("{{HttpScheme}}{{NodeHttpAddress}}/node/containerlogs/" + + "{{ContainerId}}/{{User}}/{{FileName}}?start=-4096") /* Rolled log aggregation configuration. */ diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnLogUrlSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnLogUrlSuite.scala new file mode 100644 index 0000000000000..7b32105459558 --- /dev/null +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnLogUrlSuite.scala @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.yarn + +import org.apache.spark.SparkFunSuite + +class YarnLogUrlSuite extends SparkFunSuite { + + private val testHttpScheme = "https://" + private val testNodeHttpAddress = "nodeManager:1234" + private val testContainerId = "testContainer" + private val testUser = "testUser" + private val testEnvNameToFileNameMap = Map("TEST_ENV_STDOUT" -> "stdout", + "TEST_ENV_STDERR" -> "stderr") + + test("Custom log URL - leverage all patterns, all values for patterns are available") { + val logUrlPattern = "{{HttpScheme}}{{NodeHttpAddress}}/logs/clusters/{{ClusterId}}" + + "/containers/{{ContainerId}}/users/{{User}}/files/{{FileName}}" + + val clusterId = Some("testCluster") + + val logUrls = ExecutorRunnable.buildLogUrls(logUrlPattern, testHttpScheme, testNodeHttpAddress, + clusterId, testContainerId, testUser, testEnvNameToFileNameMap) + + val expectedLogUrls = testEnvNameToFileNameMap.map { case (envName, fileName) => + envName -> (s"$testHttpScheme$testNodeHttpAddress/logs/clusters/${clusterId.get}" + + s"/containers/$testContainerId/users/$testUser/files/$fileName") + } + + assert(logUrls === expectedLogUrls) + } + + test("Custom log URL - optional pattern is not used in log URL") { + // here {{ClusterId}} is excluded in this pattern + val logUrlPattern = "{{HttpScheme}}{{NodeHttpAddress}}/logs/containers/{{ContainerId}}" + + "/users/{{User}}/files/{{FileName}}" + + // suppose the value of {{ClusterId}} pattern is not available + val clusterId = None + + // This should not throw an exception: the value for optional pattern is not available + // but we also don't use the pattern in log URL. + val logUrls = ExecutorRunnable.buildLogUrls(logUrlPattern, testHttpScheme, testNodeHttpAddress, + clusterId, testContainerId, testUser, testEnvNameToFileNameMap) + + val expectedLogUrls = testEnvNameToFileNameMap.map { case (envName, fileName) => + envName -> (s"$testHttpScheme$testNodeHttpAddress/logs/containers/$testContainerId" + + s"/users/$testUser/files/$fileName") + } + + assert(logUrls === expectedLogUrls) + } + + test("Custom log URL - optional pattern is used in log URL but the value " + + "is not present") { + // here {{ClusterId}} is included in this pattern + val logUrlPattern = "{{HttpScheme}}{{NodeHttpAddress}}/logs/clusters/{{ClusterId}}" + + "/containers/{{ContainerId}}/users/{{User}}/files/{{FileName}}" + + // suppose the value of {{ClusterId}} pattern is not available + val clusterId = None + + intercept[IllegalArgumentException] { + ExecutorRunnable.buildLogUrls(logUrlPattern, testHttpScheme, testNodeHttpAddress, + clusterId, testContainerId, testUser, testEnvNameToFileNameMap) + } + } +} From b1e08fdcf44b74e009d09150de7956e4d5a74da7 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Wed, 12 Dec 2018 07:33:05 +0900 Subject: [PATCH 03/28] Address review comments from @squito --- .../spark/deploy/yarn/ExecutorRunnable.scala | 14 +++++++------- .../apache/spark/deploy/yarn/YarnLogUrlSuite.scala | 2 +- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 95afa55d12168..012b6e5da4ff4 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -272,13 +272,13 @@ private[yarn] class ExecutorRunnable( private[yarn] object ExecutorRunnable { def buildLogUrls( - logUrlPattern: String, - httpScheme: String, - nodeHttpAddress: String, - clusterId: Option[String], - containerId: String, - user: String, - envNameToFileNameMap: Map[String, String]): Map[String, String] = { + logUrlPattern: String, + httpScheme: String, + nodeHttpAddress: String, + clusterId: Option[String], + containerId: String, + user: String, + envNameToFileNameMap: Map[String, String]): Map[String, String] = { val optionalPathVariable: Map[String, Option[String]] = Map("{{ClusterId}}" -> clusterId) val pathVariables: Map[String, String] = Map("{{HttpScheme}}" -> httpScheme, "{{NodeHttpAddress}}" -> nodeHttpAddress, diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnLogUrlSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnLogUrlSuite.scala index 7b32105459558..0eba2c97862b3 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnLogUrlSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnLogUrlSuite.scala @@ -67,7 +67,7 @@ class YarnLogUrlSuite extends SparkFunSuite { } test("Custom log URL - optional pattern is used in log URL but the value " + - "is not present") { + "is not present") { // here {{ClusterId}} is included in this pattern val logUrlPattern = "{{HttpScheme}}{{NodeHttpAddress}}/logs/clusters/{{ClusterId}}" + "/containers/{{ContainerId}}/users/{{User}}/files/{{FileName}}" From ff6d9aab4af15b5b64459f005168892fd23c1e9f Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Wed, 16 Jan 2019 16:09:50 +0900 Subject: [PATCH 04/28] Changed approach: only SHS provides custom executor log URLs --- .../deploy/history/FsHistoryProvider.scala | 2 +- .../history/HistoryAppStatusListener.scala | 93 +++++++++ .../CoarseGrainedExecutorBackend.scala | 9 +- .../spark/internal/config/History.scala | 8 + .../cluster/CoarseGrainedClusterMessage.scala | 3 +- .../CoarseGrainedSchedulerBackend.scala | 4 +- .../scheduler/cluster/ExecutorData.scala | 5 +- .../scheduler/cluster/ExecutorInfo.scala | 8 +- .../local/LocalSchedulerBackend.scala | 3 +- .../org/apache/spark/util/JsonProtocol.scala | 9 +- .../ExecutorAllocationManagerSuite.scala | 16 +- .../apache/spark/HeartbeatReceiverSuite.scala | 4 +- .../StandaloneDynamicAllocationSuite.scala | 4 +- .../history/FsHistoryProviderSuite.scala | 180 +++++++++++++++++- .../scheduler/EventLoggingListenerSuite.scala | 3 +- .../spark/status/AppStatusListenerSuite.scala | 7 +- .../apache/spark/util/JsonProtocolSuite.scala | 10 +- docs/monitoring.md | 12 ++ docs/running-on-yarn.md | 47 +++-- .../MesosFineGrainedSchedulerBackend.scala | 2 +- ...osCoarseGrainedSchedulerBackendSuite.scala | 3 +- ...esosFineGrainedSchedulerBackendSuite.scala | 15 +- .../spark/deploy/yarn/ExecutorRunnable.scala | 32 ++-- .../org/apache/spark/deploy/yarn/config.scala | 11 -- 24 files changed, 410 insertions(+), 80 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusListener.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 33e89c3936800..e198729e1cacc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -930,7 +930,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) val replayConf = conf.clone().set(ASYNC_TRACKING_ENABLED, false) val trackingStore = new ElementTrackingStore(store, replayConf) val replayBus = new ReplayListenerBus() - val listener = new AppStatusListener(trackingStore, replayConf, false, + val listener = new HistoryAppStatusListener(trackingStore, replayConf, false, lastUpdateTime = Some(lastUpdated)) replayBus.addListener(listener) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusListener.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusListener.scala new file mode 100644 index 0000000000000..85f5075945749 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusListener.scala @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.history + +import org.apache.spark.SparkConf +import org.apache.spark.internal.config.History._ +import org.apache.spark.scheduler.SparkListenerExecutorAdded +import org.apache.spark.scheduler.cluster.ExecutorInfo +import org.apache.spark.status.{AppStatusListener, AppStatusSource, ElementTrackingStore} + +private[spark] class HistoryAppStatusListener( + kvstore: ElementTrackingStore, + conf: SparkConf, + live: Boolean, + appStatusSource: Option[AppStatusSource] = None, + lastUpdateTime: Option[Long] = None) + extends AppStatusListener(kvstore, conf, live, appStatusSource, lastUpdateTime) { + + override def onExecutorAdded(event: SparkListenerExecutorAdded): Unit = { + val execInfo = event.executorInfo + val newExecInfo = new ExecutorInfo(execInfo.executorHost, execInfo.totalCores, + renewLogUrls(execInfo), execInfo.attributes) + + super.onExecutorAdded(event.copy(executorInfo = newExecInfo)) + } + + def renewLogUrls(execInfo: ExecutorInfo): Map[String, String] = { + val oldLogUrlMap = execInfo.logUrlMap + val attributes = execInfo.attributes + + conf.get(CUSTOM_EXECUTOR_LOG_URL) match { + case Some(logUrlPattern) => + val pattern = "\\{\\{([A-Za-z0-9_\\-]+)\\}\\}".r + + val allPatterns = pattern.findAllMatchIn(logUrlPattern).map(_.group(1)).toSet + val allPatternsExceptFileName = allPatterns.filter(_ != "FILE_NAME") + val allAttributeKeys = attributes.keys.toSet + val allAttributeKeysExceptLogFiles = allAttributeKeys.filter(_ != "LOG_FILES") + + if (allPatternsExceptFileName.diff(allAttributeKeysExceptLogFiles).nonEmpty) { + logFailToRenewLogUrls("some of required attributes are missing in app's event log.", + allPatternsExceptFileName, allAttributeKeys) + return oldLogUrlMap + } else if (allPatterns.contains("FILE_NAME") && !allAttributeKeys.contains("LOG_FILES")) { + logFailToRenewLogUrls("'FILE_NAME' parameter is provided, but file information is " + + "missing in app's event log.", allPatternsExceptFileName, allAttributeKeys) + return oldLogUrlMap + } + + var replacingUrl = logUrlPattern + + allPatternsExceptFileName.foreach { pattern => + // we already checked the existence of attribute when comparing keys + replacingUrl = replacingUrl.replace(s"{{$pattern}}", attributes(pattern)) + } + + if (allPatterns.contains("FILE_NAME")) { + // allAttributeKeys should contain "LOG_FILES" + attributes("LOG_FILES").split(",").map { file => + file -> replacingUrl.replace("{{FILE_NAME}}", file) + }.toMap + } else { + Map("log" -> replacingUrl) + } + + case None => oldLogUrlMap + } + } + + private def logFailToRenewLogUrls( + reason: String, + allPatterns: Set[String], + allAttributes: Set[String]): Unit = { + + logWarning(s"Fail to renew executor log urls: $reason. Required: $allPatterns / " + + s"available: $allAttributes. Failing back to show app's origin log urls.") + } +} diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 48d3630abd1f9..2865c3bc86e48 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -60,7 +60,8 @@ private[spark] class CoarseGrainedExecutorBackend( rpcEnv.asyncSetupEndpointRefByURI(driverUrl).flatMap { ref => // This is a very fast action so we can use "ThreadUtils.sameThread" driver = Some(ref) - ref.ask[Boolean](RegisterExecutor(executorId, self, hostname, cores, extractLogUrls)) + ref.ask[Boolean](RegisterExecutor(executorId, self, hostname, cores, extractLogUrls, + extractAttributes)) }(ThreadUtils.sameThread).onComplete { // This is a very fast action so we can use "ThreadUtils.sameThread" case Success(msg) => @@ -76,6 +77,12 @@ private[spark] class CoarseGrainedExecutorBackend( .map(e => (e._1.substring(prefix.length).toLowerCase(Locale.ROOT), e._2)) } + def extractAttributes: Map[String, String] = { + val prefix = "SPARK_EXECUTOR_ATTRIBUTE_" + sys.env.filterKeys(_.startsWith(prefix)) + .map(e => (e._1.substring(prefix.length).toUpperCase(Locale.ROOT), e._2)) + } + override def receive: PartialFunction[Any, Unit] = { case RegisteredExecutor => logInfo("Successfully registered with driver") diff --git a/core/src/main/scala/org/apache/spark/internal/config/History.scala b/core/src/main/scala/org/apache/spark/internal/config/History.scala index e7d25bfe33dc0..05ed848e4668d 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/History.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/History.scala @@ -125,4 +125,12 @@ private[spark] object History { val KERBEROS_KEYTAB = ConfigBuilder("spark.history.kerberos.keytab") .stringConf .createOptional + + val CUSTOM_EXECUTOR_LOG_URL = ConfigBuilder("spark.history.custom.executor.log.url") + .doc("Specifies custom spark executor log url for supporting external log service instead of " + + "using resource managers' app log urls. Spark will support some path variables via " + + "patterns which can be vary on resource managers. Please check the document for " + + "resource manager to refer which patterns are supported.") + .stringConf + .createOptional } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index 9e768c22c17e3..afb48a31754f9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -63,7 +63,8 @@ private[spark] object CoarseGrainedClusterMessages { executorRef: RpcEndpointRef, hostname: String, cores: Int, - logUrls: Map[String, String]) + logUrls: Map[String, String], + attributes: Map[String, String]) extends CoarseGrainedClusterMessage case class StatusUpdate(executorId: String, taskId: Long, state: TaskState, diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 6bc0bdd97f7d9..36209e22c072f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -183,7 +183,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { - case RegisterExecutor(executorId, executorRef, hostname, cores, logUrls) => + case RegisterExecutor(executorId, executorRef, hostname, cores, logUrls, attributes) => if (executorDataMap.contains(executorId)) { executorRef.send(RegisterExecutorFailed("Duplicate executor ID: " + executorId)) context.reply(true) @@ -207,7 +207,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp totalCoreCount.addAndGet(cores) totalRegisteredExecutors.addAndGet(1) val data = new ExecutorData(executorRef, executorAddress, hostname, - cores, cores, logUrls) + cores, cores, logUrls, attributes) // This must be synchronized because variables mutated // in this block are read when requesting executors CoarseGrainedSchedulerBackend.this.synchronized { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala index b25a4bfb501fb..ebe1c1eb0a357 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala @@ -34,5 +34,6 @@ private[cluster] class ExecutorData( override val executorHost: String, var freeCores: Int, override val totalCores: Int, - override val logUrlMap: Map[String, String] -) extends ExecutorInfo(executorHost, totalCores, logUrlMap) + override val logUrlMap: Map[String, String], + override val attributes: Map[String, String] +) extends ExecutorInfo(executorHost, totalCores, logUrlMap, attributes) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala index 7f218566146a1..8fb1be879d34c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala @@ -26,7 +26,8 @@ import org.apache.spark.annotation.DeveloperApi class ExecutorInfo( val executorHost: String, val totalCores: Int, - val logUrlMap: Map[String, String]) { + val logUrlMap: Map[String, String], + val attributes: Map[String, String]) { def canEqual(other: Any): Boolean = other.isInstanceOf[ExecutorInfo] @@ -35,12 +36,13 @@ class ExecutorInfo( (that canEqual this) && executorHost == that.executorHost && totalCores == that.totalCores && - logUrlMap == that.logUrlMap + logUrlMap == that.logUrlMap && + attributes == that.attributes case _ => false } override def hashCode(): Int = { - val state = Seq(executorHost, totalCores, logUrlMap) + val state = Seq(executorHost, totalCores, logUrlMap, attributes) state.map(_.hashCode()).foldLeft(0)((a, b) => 31 * a + b) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala index 6ff8bf29b006a..fde2a328f02f1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala @@ -129,7 +129,8 @@ private[spark] class LocalSchedulerBackend( listenerBus.post(SparkListenerExecutorAdded( System.currentTimeMillis, executorEndpoint.localExecutorId, - new ExecutorInfo(executorEndpoint.localExecutorHostname, totalCores, Map.empty))) + new ExecutorInfo(executorEndpoint.localExecutorHostname, totalCores, Map.empty, + Map.empty))) launcherBackend.setAppId(appId) launcherBackend.setState(SparkAppHandle.State.RUNNING) } diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 3370152975a58..11dcf2b5bf88d 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -486,7 +486,8 @@ private[spark] object JsonProtocol { def executorInfoToJson(executorInfo: ExecutorInfo): JValue = { ("Host" -> executorInfo.executorHost) ~ ("Total Cores" -> executorInfo.totalCores) ~ - ("Log Urls" -> mapToJson(executorInfo.logUrlMap)) + ("Log Urls" -> mapToJson(executorInfo.logUrlMap)) ~ + ("Attributes" -> mapToJson(executorInfo.attributes)) } def blockUpdatedInfoToJson(blockUpdatedInfo: BlockUpdatedInfo): JValue = { @@ -1061,7 +1062,11 @@ private[spark] object JsonProtocol { val executorHost = (json \ "Host").extract[String] val totalCores = (json \ "Total Cores").extract[Int] val logUrls = mapFromJson(json \ "Log Urls").toMap - new ExecutorInfo(executorHost, totalCores, logUrls) + val attributes = jsonOption(json \ "Attributes") match { + case Some(attr) => mapFromJson(attr).toMap + case None => Map.empty[String, String] + } + new ExecutorInfo(executorHost, totalCores, logUrls, attributes) } def blockUpdatedInfoFromJson(json: JValue): BlockUpdatedInfo = { diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index fdaea28305b45..ce0ef2e94695d 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -201,7 +201,7 @@ class ExecutorAllocationManagerSuite // Verify that running a task doesn't affect the target post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(1, 3))) post(sc.listenerBus, SparkListenerExecutorAdded( - 0L, "executor-1", new ExecutorInfo("host1", 1, Map.empty))) + 0L, "executor-1", new ExecutorInfo("host1", 1, Map.empty, Map.empty))) post(sc.listenerBus, SparkListenerTaskStart(1, 0, createTaskInfo(0, 0, "executor-1"))) assert(numExecutorsTarget(manager) === 5) assert(addExecutors(manager) === 1) @@ -809,13 +809,13 @@ class ExecutorAllocationManagerSuite // New executors have registered post(sc.listenerBus, SparkListenerExecutorAdded( - 0L, "executor-1", new ExecutorInfo("host1", 1, Map.empty))) + 0L, "executor-1", new ExecutorInfo("host1", 1, Map.empty, Map.empty))) assert(executorIds(manager).size === 1) assert(executorIds(manager).contains("executor-1")) assert(removeTimes(manager).size === 1) assert(removeTimes(manager).contains("executor-1")) post(sc.listenerBus, SparkListenerExecutorAdded( - 0L, "executor-2", new ExecutorInfo("host2", 1, Map.empty))) + 0L, "executor-2", new ExecutorInfo("host2", 1, Map.empty, Map.empty))) assert(executorIds(manager).size === 2) assert(executorIds(manager).contains("executor-2")) assert(removeTimes(manager).size === 2) @@ -842,7 +842,7 @@ class ExecutorAllocationManagerSuite post(sc.listenerBus, SparkListenerTaskStart(0, 0, createTaskInfo(0, 0, "executor-1"))) post(sc.listenerBus, SparkListenerExecutorAdded( - 0L, "executor-1", new ExecutorInfo("host1", 1, Map.empty))) + 0L, "executor-1", new ExecutorInfo("host1", 1, Map.empty, Map.empty))) assert(executorIds(manager).size === 1) assert(executorIds(manager).contains("executor-1")) assert(removeTimes(manager).size === 0) @@ -854,7 +854,7 @@ class ExecutorAllocationManagerSuite assert(executorIds(manager).isEmpty) assert(removeTimes(manager).isEmpty) post(sc.listenerBus, SparkListenerExecutorAdded( - 0L, "executor-1", new ExecutorInfo("host1", 1, Map.empty))) + 0L, "executor-1", new ExecutorInfo("host1", 1, Map.empty, Map.empty))) post(sc.listenerBus, SparkListenerTaskStart(0, 0, createTaskInfo(0, 0, "executor-1"))) assert(executorIds(manager).size === 1) @@ -862,7 +862,7 @@ class ExecutorAllocationManagerSuite assert(removeTimes(manager).size === 0) post(sc.listenerBus, SparkListenerExecutorAdded( - 0L, "executor-2", new ExecutorInfo("host1", 1, Map.empty))) + 0L, "executor-2", new ExecutorInfo("host1", 1, Map.empty, Map.empty))) assert(executorIds(manager).size === 2) assert(executorIds(manager).contains("executor-2")) assert(removeTimes(manager).size === 1) @@ -1112,7 +1112,7 @@ class ExecutorAllocationManagerSuite // test setup -- job with 2 tasks, scale up to two executors assert(numExecutorsTarget(manager) === 1) manager.listener.onExecutorAdded(SparkListenerExecutorAdded( - clock.getTimeMillis(), "executor-1", new ExecutorInfo("host1", 1, Map.empty))) + clock.getTimeMillis(), "executor-1", new ExecutorInfo("host1", 1, Map.empty, Map.empty))) manager.listener.onStageSubmitted(SparkListenerStageSubmitted(createStageInfo(0, 2))) clock.advance(1000) manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.getTimeMillis()) @@ -1120,7 +1120,7 @@ class ExecutorAllocationManagerSuite val taskInfo0 = createTaskInfo(0, 0, "executor-1") manager.listener.onTaskStart(SparkListenerTaskStart(0, 0, taskInfo0)) manager.listener.onExecutorAdded(SparkListenerExecutorAdded( - clock.getTimeMillis(), "executor-2", new ExecutorInfo("host1", 1, Map.empty))) + clock.getTimeMillis(), "executor-2", new ExecutorInfo("host1", 1, Map.empty, Map.empty))) val taskInfo1 = createTaskInfo(1, 1, "executor-2") manager.listener.onTaskStart(SparkListenerTaskStart(0, 0, taskInfo1)) assert(numExecutorsTarget(manager) === 2) diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index dbe187d54ccce..061aeb366cf5f 100644 --- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -174,9 +174,9 @@ class HeartbeatReceiverSuite val dummyExecutorEndpointRef1 = rpcEnv.setupEndpoint("fake-executor-1", dummyExecutorEndpoint1) val dummyExecutorEndpointRef2 = rpcEnv.setupEndpoint("fake-executor-2", dummyExecutorEndpoint2) fakeSchedulerBackend.driverEndpoint.askSync[Boolean]( - RegisterExecutor(executorId1, dummyExecutorEndpointRef1, "1.2.3.4", 0, Map.empty)) + RegisterExecutor(executorId1, dummyExecutorEndpointRef1, "1.2.3.4", 0, Map.empty, Map.empty)) fakeSchedulerBackend.driverEndpoint.askSync[Boolean]( - RegisterExecutor(executorId2, dummyExecutorEndpointRef2, "1.2.3.5", 0, Map.empty)) + RegisterExecutor(executorId2, dummyExecutorEndpointRef2, "1.2.3.5", 0, Map.empty, Map.empty)) heartbeatReceiverRef.askSync[Boolean](TaskSchedulerIsSet) addExecutorAndVerify(executorId1) addExecutorAndVerify(executorId2) diff --git a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala index a6d5066423d16..573a49648d780 100644 --- a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala @@ -497,7 +497,7 @@ class StandaloneDynamicAllocationSuite val endpointRef = mock(classOf[RpcEndpointRef]) val mockAddress = mock(classOf[RpcAddress]) when(endpointRef.address).thenReturn(mockAddress) - val message = RegisterExecutor("one", endpointRef, "blacklisted-host", 10, Map.empty) + val message = RegisterExecutor("one", endpointRef, "blacklisted-host", 10, Map.empty, Map.empty) // Get "localhost" on a blacklist. val taskScheduler = mock(classOf[TaskSchedulerImpl]) @@ -621,7 +621,7 @@ class StandaloneDynamicAllocationSuite val endpointRef = mock(classOf[RpcEndpointRef]) val mockAddress = mock(classOf[RpcAddress]) when(endpointRef.address).thenReturn(mockAddress) - val message = RegisterExecutor(id, endpointRef, "localhost", 10, Map.empty) + val message = RegisterExecutor(id, endpointRef, "localhost", 10, Map.empty, Map.empty) val backend = sc.schedulerBackend.asInstanceOf[CoarseGrainedSchedulerBackend] backend.driverEndpoint.askSync[Boolean](message) } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 74574e2024c10..4c4042bb3f73b 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -47,8 +47,9 @@ import org.apache.spark.internal.config.History._ import org.apache.spark.internal.config.UI.{ADMIN_ACLS, ADMIN_ACLS_GROUPS, USER_GROUPS_MAPPING} import org.apache.spark.io._ import org.apache.spark.scheduler._ +import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.security.GroupMappingServiceProvider -import org.apache.spark.status.AppStatusStore +import org.apache.spark.status.{AppStatusStore, ExecutorSummaryWrapper} import org.apache.spark.status.api.v1.{ApplicationAttemptInfo, ApplicationInfo} import org.apache.spark.util.{Clock, JsonProtocol, ManualClock, Utils} import org.apache.spark.util.logging.DriverLogger @@ -291,6 +292,160 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc } } + test("Handling executor log url without renewing") { + val conf = createTestConf() + val appId = "app1" + val user = "user1" + + val executorInfos = (1 to 5).map(createTestExecutorInfo(appId, user, _)) + + val expected: Map[ExecutorInfo, Map[String, String]] = executorInfos.map { execInfo => + execInfo -> execInfo.logUrlMap + }.toMap + + testHandlingExecutorLogUrl(conf, expected) + } + + test("Handling executor log url with custom executor url - happy case") { + val conf = createTestConf() + .set(CUSTOM_EXECUTOR_LOG_URL, "http://newhost:9999/logs/clusters/{{CLUSTER_ID}}" + + "/users/{{USER}}/containers/{{CONTAINER_ID}}/{{FILE_NAME}}") + + // some of available attributes are not used in pattern which should be OK + + val appId = "app1" + val user = "user1" + + val executorInfos = (1 to 5).map(createTestExecutorInfo(appId, user, _)) + + val expected: Map[ExecutorInfo, Map[String, String]] = executorInfos.map { execInfo => + val attr = execInfo.attributes + val newLogUrlMap = attr("LOG_FILES").split(",").map { file => + val newLogUrl = s"http://newhost:9999/logs/clusters/${attr("CLUSTER_ID")}" + + s"/users/${attr("USER")}/containers/${attr("CONTAINER_ID")}/$file" + file -> newLogUrl + }.toMap + + execInfo -> newLogUrlMap + }.toMap + + testHandlingExecutorLogUrl(conf, expected) + } + + test("Handling executor log url with custom executor url - happy case - " + + "pattern doesn't contain 'FILE_NAME'") { + val conf = createTestConf() + .set(CUSTOM_EXECUTOR_LOG_URL, "http://newhost:9999/logs/clusters/{{CLUSTER_ID}}" + + "/users/{{USER}}/containers/{{CONTAINER_ID}}") + + // some of available attributes are not used in pattern which should be OK + + val appId = "app1" + val user = "user1" + + val executorInfos = (1 to 5).map(createTestExecutorInfo(appId, user, _)) + + val expected: Map[ExecutorInfo, Map[String, String]] = executorInfos.map { execInfo => + val attr = execInfo.attributes + val newLogUrl = s"http://newhost:9999/logs/clusters/${attr("CLUSTER_ID")}" + + s"/users/${attr("USER")}/containers/${attr("CONTAINER_ID")}" + + execInfo -> Map("log" -> newLogUrl) + }.toMap + + testHandlingExecutorLogUrl(conf, expected) + } + + test("Handling executor log url with custom executor url - bad case - " + + "referring non-available attribute") { + // Here we are referring {{NON_EXISTING}} which is not available in attributes, + // which Spark will fail back to provide origin log url with warning log. + + val conf = createTestConf() + .set(CUSTOM_EXECUTOR_LOG_URL, "http://newhost:9999/logs/clusters/{{CLUSTER_ID}}" + + "/users/{{USER}}/containers/{{CONTAINER_ID}}/{{NON_EXISTING}}/{{FILE_NAME}}") + + val appId = "app1" + val user = "user1" + + val executorInfos = (1 to 5).map(createTestExecutorInfo(appId, user, _)) + + val expected: Map[ExecutorInfo, Map[String, String]] = executorInfos.map { execInfo => + + execInfo -> execInfo.logUrlMap + }.toMap + + testHandlingExecutorLogUrl(conf, expected) + } + + test("Handling executor log url with custom executor url - bad case - " + + "'FILE_NAME' is given for pattern but 'LOG_FILES' attribute is not available") { + // For this case Spark will fail back to provide origin log url with warning log. + + val conf = createTestConf() + .set(CUSTOM_EXECUTOR_LOG_URL, "http://newhost:9999/logs/clusters/{{CLUSTER_ID}}" + + "/users/{{USER}}/containers/{{CONTAINER_ID}}/{{NON_EXISTING}}/{{FILE_NAME}}") + + val appId = "app1" + val user = "user1" + + val executorInfos = (1 to 5).map( + createTestExecutorInfo(appId, user, _, includingLogFiles = false)) + + val expected: Map[ExecutorInfo, Map[String, String]] = executorInfos.map { execInfo => + execInfo -> execInfo.logUrlMap + }.toMap + + testHandlingExecutorLogUrl(conf, expected) + } + + private def testHandlingExecutorLogUrl( + conf: SparkConf, + expectedLogUrlMap: Map[ExecutorInfo, Map[String, String]]): Unit = { + val provider = new FsHistoryProvider(conf) + + val attempt1 = newLogFile("app1", Some("attempt1"), inProgress = true) + + val executorAddedEvents = expectedLogUrlMap.keys.zipWithIndex.map { case (execInfo, idx) => + val event = SparkListenerExecutorAdded(1 + idx, s"exec$idx", execInfo) + event + }.toList.sortBy(_.time) + val allEvents = List(SparkListenerApplicationStart("app1", Some("app1"), 1L, + "test", Some("attempt1"))) ++ executorAddedEvents + + writeFile(attempt1, true, None, allEvents: _*) + + updateAndCheck(provider) { list => + list.size should be (1) + list.head.attempts.size should be (1) + + list.foreach { app => + app.attempts.foreach { attempt => + val appUi = provider.getAppUI(app.id, attempt.attemptId) + appUi should not be null + val executors = appUi.get.ui.store.store.view(classOf[ExecutorSummaryWrapper]) + .closeableIterator().asScala + executors should not be null + + val iterForExpectation = expectedLogUrlMap.iterator + + var executorCount = 0 + while (executors.hasNext) { + val executor = executors.next() + val expectation = iterForExpectation.next() + + executor.host should be(expectation._1.executorHost) + executor.info.executorLogs should be(expectation._2) + + executorCount += 1 + } + + executorCount should be (expectedLogUrlMap.size) + } + } + } + } + test("log cleaner") { val maxAge = TimeUnit.SECONDS.toMillis(10) val clock = new ManualClock(maxAge / 2) @@ -1048,6 +1203,29 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc conf } + private def createTestExecutorInfo( + appId: String, + user: String, + executorSeqNum: Int, + includingLogFiles: Boolean = true): ExecutorInfo = { + val host = s"host$executorSeqNum" + val container = s"container$executorSeqNum" + val cluster = s"cluster$executorSeqNum" + val logUrlPrefix = s"http://$host:8888/$appId/$container/origin" + + val executorLogUrlMap = Map("stdout" -> s"$logUrlPrefix/stdout", + "stderr" -> s"$logUrlPrefix/stderr") + + val executorAttributes = if (includingLogFiles) { + Map("LOG_FILES" -> "stdout,stderr", "CONTAINER_ID" -> container, + "CLUSTER_ID" -> cluster, "USER" -> user) + } else { + Map("CONTAINER_ID" -> container, "CLUSTER_ID" -> cluster, "USER" -> user) + } + + new ExecutorInfo(host, 1, executorLogUrlMap, executorAttributes) + } + private class SafeModeTestProvider(conf: SparkConf, clock: Clock) extends FsHistoryProvider(conf, clock) { 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 811b9fe46fdf6..40521f0732177 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -422,7 +422,8 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit } private def createExecutorAddedEvent(executorId: Int) = { - SparkListenerExecutorAdded(0L, executorId.toString, new ExecutorInfo("host1", 1, Map.empty)) + SparkListenerExecutorAdded(0L, executorId.toString, + new ExecutorInfo("host1", 1, Map.empty, Map.empty)) } private def createExecutorRemovedEvent(executorId: Int) = { diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala index ede9466b3d63c..356e6d165190a 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala @@ -135,7 +135,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { execIds.foreach { id => listener.onExecutorAdded(SparkListenerExecutorAdded(time, id, - new ExecutorInfo(s"$id.example.com", 1, Map()))) + new ExecutorInfo(s"$id.example.com", 1, Map.empty, Map.empty))) } execIds.foreach { id => @@ -685,7 +685,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { val bm2 = BlockManagerId("2", "2.example.com", 84) Seq(bm1, bm2).foreach { bm => listener.onExecutorAdded(SparkListenerExecutorAdded(1L, bm.executorId, - new ExecutorInfo(bm.host, 1, Map()))) + new ExecutorInfo(bm.host, 1, Map.empty, Map.empty))) listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm, maxMemory)) check[ExecutorSummaryWrapper](bm.executorId) { exec => assert(exec.info.maxMemory === maxMemory) @@ -1553,7 +1553,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { /** Create an executor added event for the specified executor Id. */ private def createExecutorAddedEvent(executorId: Int) = { - SparkListenerExecutorAdded(0L, executorId.toString, new ExecutorInfo("host1", 1, Map.empty)) + SparkListenerExecutorAdded(0L, executorId.toString, + new ExecutorInfo("host1", 1, Map.empty, Map.empty)) } /** Create an executor added event for the specified executor Id. */ 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 c63f04db184d4..c3ff379c84fff 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -76,13 +76,14 @@ class JsonProtocolSuite extends SparkFunSuite { BlockManagerId("Scarce", "to be counted...", 100)) val unpersistRdd = SparkListenerUnpersistRDD(12345) val logUrlMap = Map("stderr" -> "mystderr", "stdout" -> "mystdout").toMap + val attributes = Map("ContainerId" -> "ct1", "User" -> "spark").toMap val applicationStart = SparkListenerApplicationStart("The winner of all", Some("appId"), 42L, "Garfield", Some("appAttempt")) val applicationStartWithLogs = SparkListenerApplicationStart("The winner of all", Some("appId"), 42L, "Garfield", Some("appAttempt"), Some(logUrlMap)) val applicationEnd = SparkListenerApplicationEnd(42L) val executorAdded = SparkListenerExecutorAdded(executorAddedTime, "exec1", - new ExecutorInfo("Hostee.awesome.com", 11, logUrlMap)) + new ExecutorInfo("Hostee.awesome.com", 11, logUrlMap, attributes)) val executorRemoved = SparkListenerExecutorRemoved(executorRemovedTime, "exec2", "test reason") val executorBlacklisted = SparkListenerExecutorBlacklisted(executorBlacklistedTime, "exec1", 22) val executorUnblacklisted = @@ -138,13 +139,14 @@ class JsonProtocolSuite extends SparkFunSuite { test("Dependent Classes") { val logUrlMap = Map("stderr" -> "mystderr", "stdout" -> "mystdout").toMap + val attributes = Map("ContainerId" -> "ct1", "User" -> "spark").toMap testRDDInfo(makeRddInfo(2, 3, 4, 5L, 6L)) testStageInfo(makeStageInfo(10, 20, 30, 40L, 50L)) testTaskInfo(makeTaskInfo(999L, 888, 55, 777L, false)) testTaskMetrics(makeTaskMetrics( 33333L, 44444L, 55555L, 66666L, 7, 8, hasHadoopInput = false, hasOutput = false)) testBlockManagerId(BlockManagerId("Hong", "Kong", 500)) - testExecutorInfo(new ExecutorInfo("host", 43, logUrlMap)) + testExecutorInfo(new ExecutorInfo("host", 43, logUrlMap, attributes)) // StorageLevel testStorageLevel(StorageLevel.NONE) @@ -1858,6 +1860,10 @@ private[spark] object JsonProtocolSuite extends Assertions { | "Log Urls" : { | "stderr" : "mystderr", | "stdout" : "mystdout" + | }, + | "Attributes" : { + | "ContainerId" : "ct1", + | "User" : "spark" | } | } |} diff --git a/docs/monitoring.md b/docs/monitoring.md index 6bb620a2e5f69..f79f5de379e4a 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -248,6 +248,18 @@ Security options for the Spark History Server are covered more detail in the written to disk will be re-used in the event of a history server restart. + + spark.history.custom.executor.log.url + (none) + + Specifies custom spark executor log url for supporting external log service instead of using resource managers' app log urls. + Spark will support some path variables via patterns which can be vary on resource managers. + Please check the document for resource manager to refer which patterns are supported. +

+ For now, YARN mode only supports this configuration. + + + Note that in all of these UIs, the tables are sortable by clicking their headers, diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index d2c6af85103f8..036cc8fd1599b 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -430,21 +430,6 @@ To use a custom metrics.properties for the application master and executors, upd See spark.yarn.config.gatewayPath. - - spark.yarn.custom.log.url - {{HttpScheme}}{{NodeHttpAddress}}/node/containerlogs/{{ContainerId}}/{{User}}/{{FileName}}?start=-4096 - - Specifies custom spark log url for supporting external log service rather than NodeManager webapp address. - Spark will support some path variables via patterns. Supported patterns and allocated values are below:

-

- * `{{HttpScheme}}`: `http://`/`https://` according to YARN HTTP policy. (Configured via `yarn.http.policy`)

- * `{{NodeHttpAddress}}`: The "host:port" of node where container was run.

- * `{{ClusterId}}`: The cluster ID of Resource Manager. (Configured via `yarn.resourcemanager.cluster-id`)

- * `{{ContainerId}}`: The ID of container.

- * `{{User}}`: `SPARK_USER` on system environment.

- * `{{FileName}}`: `stdout`/`stderr`.

- - spark.yarn.rolledLog.includePattern (none) @@ -487,6 +472,36 @@ To use a custom metrics.properties for the application master and executors, upd +#### Available patterns for SHS custom executor log URL + + + + + + + + + + + + + + + + + + + + + + + + + + + +
PatternMeaning
{{HTTP_SCHEME}}`http://`/`https://` according to YARN HTTP policy. (Configured via `yarn.http.policy`)
{{NODE_HTTP_ADDRESS}}The "host:port" of node where container was run.
{{CLUSTER_ID}}The cluster ID of Resource Manager. (Configured via `yarn.resourcemanager.cluster-id`)
{{CONTAINER_ID}}The ID of container.
{{USER}}'SPARK_USER' on system environment.
{{FILE_NAME}}`stdout`, `stderr`.
+ # Important notes - Whether core requests are honored in scheduling decisions depends on which scheduler is in use and how it is configured. @@ -674,4 +689,4 @@ do the following: - On the Spark History Server, add org.apache.spark.deploy.yarn.YarnProxyRedirectFilter to the list of filters in the spark.ui.filters configuration. -Be aware that the history server information may not be up-to-date with the application's state. +Be aware that the history server information may not be up-to-date with the application's state. \ No newline at end of file diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala index 192f9407a1ba4..c5c78422023b6 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackend.scala @@ -328,7 +328,7 @@ private[spark] class MesosFineGrainedSchedulerBackend( slaveIdToWorkerOffer.get(slaveId).foreach(o => listenerBus.post(SparkListenerExecutorAdded(System.currentTimeMillis(), slaveId, // TODO: Add support for log urls for Mesos - new ExecutorInfo(o.host, o.cores, Map.empty))) + new ExecutorInfo(o.host, o.cores, Map.empty, Map.empty))) ) logTrace(s"Launching Mesos tasks on slave '$slaveId', tasks:\n${getTasksSummary(tasks)}") d.launchTasks(Collections.singleton(slaveIdToOffer(slaveId).getId), tasks, filters) diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala index f2fd5e606b6c1..a4e9cc3f5d358 100644 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala @@ -687,7 +687,8 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite private def registerMockExecutor(executorId: String, slaveId: String, cores: Integer) = { val mockEndpointRef = mock[RpcEndpointRef] val mockAddress = mock[RpcAddress] - val message = RegisterExecutor(executorId, mockEndpointRef, slaveId, cores, Map.empty) + val message = RegisterExecutor(executorId, mockEndpointRef, slaveId, cores, Map.empty, + Map.empty) backend.driverEndpoint.askSync[Boolean](message) } diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala index c9b7e6c439c4b..0b6d93f62d408 100644 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala @@ -86,7 +86,8 @@ class MesosFineGrainedSchedulerBackendSuite val listenerBus = mock[LiveListenerBus] listenerBus.post( - SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty))) + SparkListenerExecutorAdded(anyLong, "s1", + new ExecutorInfo("host1", 2, Map.empty, Map.empty))) val sc = mock[SparkContext] when(sc.getSparkHome()).thenReturn(Option("/spark-home")) @@ -117,7 +118,8 @@ class MesosFineGrainedSchedulerBackendSuite val listenerBus = mock[LiveListenerBus] listenerBus.post( - SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty))) + SparkListenerExecutorAdded(anyLong, "s1", + new ExecutorInfo("host1", 2, Map.empty, Map.empty))) val sc = mock[SparkContext] when(sc.getSparkHome()).thenReturn(Option("/spark-home")) @@ -157,7 +159,8 @@ class MesosFineGrainedSchedulerBackendSuite val listenerBus = mock[LiveListenerBus] listenerBus.post( - SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty))) + SparkListenerExecutorAdded(anyLong, "s1", + new ExecutorInfo("host1", 2, Map.empty, Map.empty))) val sc = mock[SparkContext] when(sc.executorMemory).thenReturn(100) @@ -217,7 +220,8 @@ class MesosFineGrainedSchedulerBackendSuite val listenerBus = mock[LiveListenerBus] listenerBus.post( - SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty))) + SparkListenerExecutorAdded(anyLong, "s1", + new ExecutorInfo("host1", 2, Map.empty, Map.empty))) val sc = mock[SparkContext] when(sc.executorMemory).thenReturn(100) @@ -308,7 +312,8 @@ class MesosFineGrainedSchedulerBackendSuite val listenerBus = mock[LiveListenerBus] listenerBus.post( - SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty))) + SparkListenerExecutorAdded(anyLong, "s1", + new ExecutorInfo("host1", 2, Map.empty, Map.empty))) val sc = mock[SparkContext] when(sc.executorMemory).thenReturn(100) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 012b6e5da4ff4..f0bc9420e1ad9 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -248,21 +248,24 @@ private[yarn] class ExecutorRunnable( } } - // Add log urls + // Add log urls, as well as executor attributes container.foreach { c => - sys.env.filterKeys(_.endsWith("USER")).foreach { user => - val containerId = ConverterUtils.toString(c.getId) - val address = c.getNodeHttpAddress - - val customLogUrl = sparkConf.get(config.CUSTOM_LOG_URL) - - val envNameToFileNameMap = Map("SPARK_LOG_URL_STDERR" -> "stderr", - "SPARK_LOG_URL_STDOUT" -> "stdout") - val logUrls = ExecutorRunnable.buildLogUrls(customLogUrl, httpScheme, address, - clusterId, containerId, user, envNameToFileNameMap) - logUrls.foreach { case (envName, url) => - env(envName) = url - } + val containerId = ConverterUtils.toString(c.getId) + val address = c.getNodeHttpAddress + + env("SPARK_EXECUTOR_ATTRIBUTE_HTTP_SCHEME") = httpScheme + env("SPARK_EXECUTOR_ATTRIBUTE_NODE_HTTP_ADDRESS") = address + env("SPARK_EXECUTOR_ATTRIBUTE_CLUSTER_ID") = clusterId.getOrElse("") + env("SPARK_EXECUTOR_ATTRIBUTE_CONTAINER_ID") = containerId + + sys.env.filterKeys(_.endsWith("USER")).foreach { u => + val user = u._2 + val baseUrl = s"$httpScheme$address/node/containerlogs/$containerId/$user" + env("SPARK_LOG_URL_STDERR") = s"$baseUrl/stderr?start=-4096" + env("SPARK_LOG_URL_STDOUT") = s"$baseUrl/stdout?start=-4096" + + env("SPARK_EXECUTOR_ATTRIBUTE_USER") = user + env("SPARK_EXECUTOR_ATTRIBUTE_LOG_FILES") = "stderr,stdout" } } @@ -271,6 +274,7 @@ private[yarn] class ExecutorRunnable( } private[yarn] object ExecutorRunnable { + // FIXME: MOVE TO SHS side, as well as allowing arbitrary parameters def buildLogUrls( logUrlPattern: String, httpScheme: String, diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala index 7214b09f772b0..16adaec04802e 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala @@ -258,17 +258,6 @@ package object config { "that hosts fs.defaultFS does not need to be listed here.") .fallbackConf(NAMENODES_TO_ACCESS) - /* YARN log url configuration. */ - private[spark] val CUSTOM_LOG_URL = ConfigBuilder("spark.yarn.custom.log.url") - .doc("Specifies custom spark log url for external log service other than node http address. " + - "Spark will support some path variables via patterns. " + - "Supported patterns are 1. `{{HttpScheme}}` 2. `{{NodeHttpAddress}}` 3. `{{ClusterId}}` " + - "4. `{{ContainerId}}` 5. `{{User}}` 6. `{{FileName}}`. The default value is pointing to " + - "NodeManager's container log URL.") - .stringConf - .createWithDefault("{{HttpScheme}}{{NodeHttpAddress}}/node/containerlogs/" + - "{{ContainerId}}/{{User}}/{{FileName}}?start=-4096") - /* Rolled log aggregation configuration. */ private[spark] val ROLLED_LOG_INCLUDE_PATTERN = From a36dd58e35006fa0e9f3e81a571015f41074f24b Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Thu, 17 Jan 2019 12:59:13 +0900 Subject: [PATCH 05/28] Fix binary compatibility issue --- .../org/apache/spark/scheduler/cluster/ExecutorInfo.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala index 8fb1be879d34c..3197e06fcd13a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala @@ -29,6 +29,10 @@ class ExecutorInfo( val logUrlMap: Map[String, String], val attributes: Map[String, String]) { + def this(executorHost: String, totalCores: Int, logUrlMap: Map[String, String]) = { + this(executorHost, totalCores, logUrlMap, Map.empty) + } + def canEqual(other: Any): Boolean = other.isInstanceOf[ExecutorInfo] override def equals(other: Any): Boolean = other match { From 665f0b3473c08ecca745c6356febd67e6188a47f Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Thu, 17 Jan 2019 15:51:23 +0900 Subject: [PATCH 06/28] Clean up old stuff --- .../spark/deploy/yarn/ExecutorRunnable.scala | 39 --------- .../spark/deploy/yarn/YarnLogUrlSuite.scala | 83 ------------------- 2 files changed, 122 deletions(-) delete mode 100644 resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnLogUrlSuite.scala diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index f0bc9420e1ad9..74c30f5211e1f 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -272,42 +272,3 @@ private[yarn] class ExecutorRunnable( env } } - -private[yarn] object ExecutorRunnable { - // FIXME: MOVE TO SHS side, as well as allowing arbitrary parameters - def buildLogUrls( - logUrlPattern: String, - httpScheme: String, - nodeHttpAddress: String, - clusterId: Option[String], - containerId: String, - user: String, - envNameToFileNameMap: Map[String, String]): Map[String, String] = { - val optionalPathVariable: Map[String, Option[String]] = Map("{{ClusterId}}" -> clusterId) - val pathVariables: Map[String, String] = Map("{{HttpScheme}}" -> httpScheme, - "{{NodeHttpAddress}}" -> nodeHttpAddress, - "{{ContainerId}}" -> containerId, - "{{User}}" -> user) - - var replacingUrl = logUrlPattern - - optionalPathVariable.foreach { - case (pattern, Some(value)) if replacingUrl.contains(pattern) => - replacingUrl = replacingUrl.replace(pattern, value) - - case (pattern, None) if replacingUrl.contains(pattern) => - throw new IllegalArgumentException(s"Pattern $pattern is specified in " + - s"log URL, but Spark can't retrieve the value. Please check relevant YARN config.") - - case _ => - } - - pathVariables.foreach { case (pattern, value) => - replacingUrl = replacingUrl.replace(pattern, value) - } - - envNameToFileNameMap.map { case (envName, fileName) => - envName -> replacingUrl.replace("{{FileName}}", fileName) - } - } -} diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnLogUrlSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnLogUrlSuite.scala deleted file mode 100644 index 0eba2c97862b3..0000000000000 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnLogUrlSuite.scala +++ /dev/null @@ -1,83 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.yarn - -import org.apache.spark.SparkFunSuite - -class YarnLogUrlSuite extends SparkFunSuite { - - private val testHttpScheme = "https://" - private val testNodeHttpAddress = "nodeManager:1234" - private val testContainerId = "testContainer" - private val testUser = "testUser" - private val testEnvNameToFileNameMap = Map("TEST_ENV_STDOUT" -> "stdout", - "TEST_ENV_STDERR" -> "stderr") - - test("Custom log URL - leverage all patterns, all values for patterns are available") { - val logUrlPattern = "{{HttpScheme}}{{NodeHttpAddress}}/logs/clusters/{{ClusterId}}" + - "/containers/{{ContainerId}}/users/{{User}}/files/{{FileName}}" - - val clusterId = Some("testCluster") - - val logUrls = ExecutorRunnable.buildLogUrls(logUrlPattern, testHttpScheme, testNodeHttpAddress, - clusterId, testContainerId, testUser, testEnvNameToFileNameMap) - - val expectedLogUrls = testEnvNameToFileNameMap.map { case (envName, fileName) => - envName -> (s"$testHttpScheme$testNodeHttpAddress/logs/clusters/${clusterId.get}" + - s"/containers/$testContainerId/users/$testUser/files/$fileName") - } - - assert(logUrls === expectedLogUrls) - } - - test("Custom log URL - optional pattern is not used in log URL") { - // here {{ClusterId}} is excluded in this pattern - val logUrlPattern = "{{HttpScheme}}{{NodeHttpAddress}}/logs/containers/{{ContainerId}}" + - "/users/{{User}}/files/{{FileName}}" - - // suppose the value of {{ClusterId}} pattern is not available - val clusterId = None - - // This should not throw an exception: the value for optional pattern is not available - // but we also don't use the pattern in log URL. - val logUrls = ExecutorRunnable.buildLogUrls(logUrlPattern, testHttpScheme, testNodeHttpAddress, - clusterId, testContainerId, testUser, testEnvNameToFileNameMap) - - val expectedLogUrls = testEnvNameToFileNameMap.map { case (envName, fileName) => - envName -> (s"$testHttpScheme$testNodeHttpAddress/logs/containers/$testContainerId" + - s"/users/$testUser/files/$fileName") - } - - assert(logUrls === expectedLogUrls) - } - - test("Custom log URL - optional pattern is used in log URL but the value " + - "is not present") { - // here {{ClusterId}} is included in this pattern - val logUrlPattern = "{{HttpScheme}}{{NodeHttpAddress}}/logs/clusters/{{ClusterId}}" + - "/containers/{{ContainerId}}/users/{{User}}/files/{{FileName}}" - - // suppose the value of {{ClusterId}} pattern is not available - val clusterId = None - - intercept[IllegalArgumentException] { - ExecutorRunnable.buildLogUrls(logUrlPattern, testHttpScheme, testNodeHttpAddress, - clusterId, testContainerId, testUser, testEnvNameToFileNameMap) - } - } -} From 8cd1db3c7305e4ec1799e4a23c71b9e81668bfea Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Fri, 18 Jan 2019 07:55:54 +0900 Subject: [PATCH 07/28] Address review comments from @squito --- .../history/HistoryAppStatusListener.scala | 16 +++++++--------- .../deploy/history/FsHistoryProviderSuite.scala | 9 +++------ 2 files changed, 10 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusListener.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusListener.scala index 85f5075945749..a8de2fe4ac259 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusListener.scala @@ -34,12 +34,12 @@ private[spark] class HistoryAppStatusListener( override def onExecutorAdded(event: SparkListenerExecutorAdded): Unit = { val execInfo = event.executorInfo val newExecInfo = new ExecutorInfo(execInfo.executorHost, execInfo.totalCores, - renewLogUrls(execInfo), execInfo.attributes) + replaceLogUrls(execInfo), execInfo.attributes) super.onExecutorAdded(event.copy(executorInfo = newExecInfo)) } - def renewLogUrls(execInfo: ExecutorInfo): Map[String, String] = { + def replaceLogUrls(execInfo: ExecutorInfo): Map[String, String] = { val oldLogUrlMap = execInfo.logUrlMap val attributes = execInfo.attributes @@ -49,7 +49,7 @@ private[spark] class HistoryAppStatusListener( val allPatterns = pattern.findAllMatchIn(logUrlPattern).map(_.group(1)).toSet val allPatternsExceptFileName = allPatterns.filter(_ != "FILE_NAME") - val allAttributeKeys = attributes.keys.toSet + val allAttributeKeys = attributes.keySet val allAttributeKeysExceptLogFiles = allAttributeKeys.filter(_ != "LOG_FILES") if (allPatternsExceptFileName.diff(allAttributeKeysExceptLogFiles).nonEmpty) { @@ -62,20 +62,18 @@ private[spark] class HistoryAppStatusListener( return oldLogUrlMap } - var replacingUrl = logUrlPattern - - allPatternsExceptFileName.foreach { pattern => + val updatedUrl = allPatternsExceptFileName.foldLeft(logUrlPattern) { case( orig, patt) => // we already checked the existence of attribute when comparing keys - replacingUrl = replacingUrl.replace(s"{{$pattern}}", attributes(pattern)) + orig.replace(s"{{$patt}}", attributes(patt)) } if (allPatterns.contains("FILE_NAME")) { // allAttributeKeys should contain "LOG_FILES" attributes("LOG_FILES").split(",").map { file => - file -> replacingUrl.replace("{{FILE_NAME}}", file) + file -> updatedUrl.replace("{{FILE_NAME}}", file) }.toMap } else { - Map("log" -> replacingUrl) + Map("log" -> updatedUrl) } case None => oldLogUrlMap diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 4c4042bb3f73b..f6f080e87e331 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -1216,12 +1216,9 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc val executorLogUrlMap = Map("stdout" -> s"$logUrlPrefix/stdout", "stderr" -> s"$logUrlPrefix/stderr") - val executorAttributes = if (includingLogFiles) { - Map("LOG_FILES" -> "stdout,stderr", "CONTAINER_ID" -> container, - "CLUSTER_ID" -> cluster, "USER" -> user) - } else { - Map("CONTAINER_ID" -> container, "CLUSTER_ID" -> cluster, "USER" -> user) - } + val extraAttributes = if (includingLogFiles) Map("LOG_FILES" -> "stdout,stderr") else Map.empty + val executorAttributes = Map("CONTAINER_ID" -> container, "CLUSTER_ID" -> cluster, + "USER" -> user) ++ extraAttributes new ExecutorInfo(host, 1, executorLogUrlMap, executorAttributes) } From b641bd1de5d9fdf7f896c7c6369f651abbb94006 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Fri, 18 Jan 2019 09:50:46 +0900 Subject: [PATCH 08/28] Move the role of replacing log URLs to AppStatusStore - HistoryAppStatusStore This avoids custom log URLs being stored which could prevent applying new configuration --- .../deploy/history/FsHistoryProvider.scala | 9 +- .../history/HistoryAppStatusListener.scala | 91 ---------------- .../history/HistoryAppStatusStore.scala | 102 ++++++++++++++++++ .../spark/status/AppStatusListener.scala | 1 + .../org/apache/spark/status/LiveEntity.scala | 4 +- .../org/apache/spark/status/api/v1/api.scala | 57 +++++----- .../history/FsHistoryProviderSuite.scala | 7 +- 7 files changed, 142 insertions(+), 129 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusListener.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index e198729e1cacc..56aea0c53d497 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -359,10 +359,9 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) return None } - val ui = SparkUI.create(None, new AppStatusStore(kvstore), conf, secManager, app.info.name, - HistoryServer.getAttemptURI(appId, attempt.info.attemptId), - attempt.info.startTime.getTime(), - attempt.info.appSparkVersion) + val ui = SparkUI.create(None, new HistoryAppStatusStore(conf, kvstore), conf, secManager, + app.info.name, HistoryServer.getAttemptURI(appId, attempt.info.attemptId), + attempt.info.startTime.getTime(), attempt.info.appSparkVersion) loadPlugins().foreach(_.setupUI(ui)) val loadedUI = LoadedAppUI(ui) @@ -930,7 +929,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) val replayConf = conf.clone().set(ASYNC_TRACKING_ENABLED, false) val trackingStore = new ElementTrackingStore(store, replayConf) val replayBus = new ReplayListenerBus() - val listener = new HistoryAppStatusListener(trackingStore, replayConf, false, + val listener = new AppStatusListener(trackingStore, replayConf, false, lastUpdateTime = Some(lastUpdated)) replayBus.addListener(listener) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusListener.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusListener.scala deleted file mode 100644 index a8de2fe4ac259..0000000000000 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusListener.scala +++ /dev/null @@ -1,91 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.history - -import org.apache.spark.SparkConf -import org.apache.spark.internal.config.History._ -import org.apache.spark.scheduler.SparkListenerExecutorAdded -import org.apache.spark.scheduler.cluster.ExecutorInfo -import org.apache.spark.status.{AppStatusListener, AppStatusSource, ElementTrackingStore} - -private[spark] class HistoryAppStatusListener( - kvstore: ElementTrackingStore, - conf: SparkConf, - live: Boolean, - appStatusSource: Option[AppStatusSource] = None, - lastUpdateTime: Option[Long] = None) - extends AppStatusListener(kvstore, conf, live, appStatusSource, lastUpdateTime) { - - override def onExecutorAdded(event: SparkListenerExecutorAdded): Unit = { - val execInfo = event.executorInfo - val newExecInfo = new ExecutorInfo(execInfo.executorHost, execInfo.totalCores, - replaceLogUrls(execInfo), execInfo.attributes) - - super.onExecutorAdded(event.copy(executorInfo = newExecInfo)) - } - - def replaceLogUrls(execInfo: ExecutorInfo): Map[String, String] = { - val oldLogUrlMap = execInfo.logUrlMap - val attributes = execInfo.attributes - - conf.get(CUSTOM_EXECUTOR_LOG_URL) match { - case Some(logUrlPattern) => - val pattern = "\\{\\{([A-Za-z0-9_\\-]+)\\}\\}".r - - val allPatterns = pattern.findAllMatchIn(logUrlPattern).map(_.group(1)).toSet - val allPatternsExceptFileName = allPatterns.filter(_ != "FILE_NAME") - val allAttributeKeys = attributes.keySet - val allAttributeKeysExceptLogFiles = allAttributeKeys.filter(_ != "LOG_FILES") - - if (allPatternsExceptFileName.diff(allAttributeKeysExceptLogFiles).nonEmpty) { - logFailToRenewLogUrls("some of required attributes are missing in app's event log.", - allPatternsExceptFileName, allAttributeKeys) - return oldLogUrlMap - } else if (allPatterns.contains("FILE_NAME") && !allAttributeKeys.contains("LOG_FILES")) { - logFailToRenewLogUrls("'FILE_NAME' parameter is provided, but file information is " + - "missing in app's event log.", allPatternsExceptFileName, allAttributeKeys) - return oldLogUrlMap - } - - val updatedUrl = allPatternsExceptFileName.foldLeft(logUrlPattern) { case( orig, patt) => - // we already checked the existence of attribute when comparing keys - orig.replace(s"{{$patt}}", attributes(patt)) - } - - if (allPatterns.contains("FILE_NAME")) { - // allAttributeKeys should contain "LOG_FILES" - attributes("LOG_FILES").split(",").map { file => - file -> updatedUrl.replace("{{FILE_NAME}}", file) - }.toMap - } else { - Map("log" -> updatedUrl) - } - - case None => oldLogUrlMap - } - } - - private def logFailToRenewLogUrls( - reason: String, - allPatterns: Set[String], - allAttributes: Set[String]): Unit = { - - logWarning(s"Fail to renew executor log urls: $reason. Required: $allPatterns / " + - s"available: $allAttributes. Failing back to show app's origin log urls.") - } -} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala new file mode 100644 index 0000000000000..11c1396a417d5 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.history + +import scala.util.matching.Regex + +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.History.CUSTOM_EXECUTOR_LOG_URL +import org.apache.spark.status.{AppStatusListener, AppStatusStore} +import org.apache.spark.status.api.v1 +import org.apache.spark.status.api.v1.ExecutorSummary +import org.apache.spark.util.kvstore.KVStore + +private[spark] class HistoryAppStatusStore( + conf: SparkConf, + store: KVStore, + listener: Option[AppStatusListener] = None) + extends AppStatusStore(store, listener) with Logging { + + import HistoryAppStatusStore._ + + private val logUrlPattern: Option[String] = conf.get(CUSTOM_EXECUTOR_LOG_URL) + + override def executorList(activeOnly: Boolean): Seq[v1.ExecutorSummary] = { + logUrlPattern match { + case Some(pattern) => super.executorList(activeOnly).map(replaceLogUrls(_, pattern)) + case None => super.executorList(activeOnly) + } + } + + override def executorSummary(executorId: String): v1.ExecutorSummary = { + logUrlPattern match { + case Some(pattern) => replaceLogUrls(super.executorSummary(executorId), pattern) + case None => super.executorSummary(executorId) + } + } + + private def replaceLogUrls(exec: ExecutorSummary, urlPattern: String): ExecutorSummary = { + val attributes = exec.attributes + + val allPatterns = CUSTOM_URL_PATTERN_REGEX.findAllMatchIn(urlPattern).map(_.group(1)).toSet + val allPatternsExceptFileName = allPatterns.filter(_ != "FILE_NAME") + val allAttributeKeys = attributes.keySet + val allAttributeKeysExceptLogFiles = allAttributeKeys.filter(_ != "LOG_FILES") + + if (allPatternsExceptFileName.diff(allAttributeKeysExceptLogFiles).nonEmpty) { + logFailToRenewLogUrls("some of required attributes are missing in app's event log.", + allPatternsExceptFileName, allAttributeKeys) + return exec + } else if (allPatterns.contains("FILE_NAME") && !allAttributeKeys.contains("LOG_FILES")) { + logFailToRenewLogUrls("'FILE_NAME' parameter is provided, but file information is " + + "missing in app's event log.", allPatternsExceptFileName, allAttributeKeys) + return exec + } + + val updatedUrl = allPatternsExceptFileName.foldLeft(urlPattern) { case(orig, patt) => + // we already checked the existence of attribute when comparing keys + orig.replace(s"{{$patt}}", attributes(patt)) + } + + val newLogUrlMap = if (allPatterns.contains("FILE_NAME")) { + // allAttributeKeys should contain "LOG_FILES" + attributes("LOG_FILES").split(",").map { file => + file -> updatedUrl.replace("{{FILE_NAME}}", file) + }.toMap + } else { + Map("log" -> updatedUrl) + } + + exec.copy(executorLogs = newLogUrlMap) + } + + private def logFailToRenewLogUrls( + reason: String, + allPatterns: Set[String], + allAttributes: Set[String]): Unit = { + + logWarning(s"Fail to renew executor log urls: $reason. Required: $allPatterns / " + + s"available: $allAttributes. Failing back to show app's origin log urls.") + } + +} + +object HistoryAppStatusStore { + val CUSTOM_URL_PATTERN_REGEX: Regex = "\\{\\{([A-Za-z0-9_\\-]+)\\}\\}".r +} diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala index 1067cdc84ceb2..ce6e1f7b11b8e 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -190,6 +190,7 @@ private[spark] class AppStatusListener( exec.totalCores = event.executorInfo.totalCores exec.maxTasks = event.executorInfo.totalCores / coresPerTask exec.executorLogs = event.executorInfo.logUrlMap + exec.attributes = event.executorInfo.attributes liveUpdate(exec, System.nanoTime()) } diff --git a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala index 7f7b83a54d794..6d7b34ae979f2 100644 --- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala +++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala @@ -258,6 +258,7 @@ private class LiveExecutor(val executorId: String, _addTime: Long) extends LiveE var blacklistedInStages: Set[Int] = TreeSet() var executorLogs = Map[String, String]() + var attributes = Map[String, String]() // Memory metrics. They may not be recorded (e.g. old event logs) so if totalOnHeap is not // initialized, the store will not contain this information. @@ -306,7 +307,8 @@ private class LiveExecutor(val executorId: String, _addTime: Long) extends LiveE executorLogs, memoryMetrics, blacklistedInStages, - Some(peakExecutorMetrics).filter(_.isSet)) + Some(peakExecutorMetrics).filter(_.isSet), + attributes) new ExecutorSummaryWrapper(info) } } diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala index 825fc54278069..8424b9636882c 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala @@ -21,7 +21,7 @@ import java.util.Date import scala.xml.{NodeSeq, Text} -import com.fasterxml.jackson.annotation.JsonIgnoreProperties +import com.fasterxml.jackson.annotation.{JsonIgnore, JsonIgnoreProperties} import com.fasterxml.jackson.core.{JsonGenerator, JsonParser} import com.fasterxml.jackson.core.`type`.TypeReference import com.fasterxml.jackson.databind.{DeserializationContext, JsonDeserializer, JsonSerializer, SerializerProvider} @@ -78,35 +78,36 @@ class ExecutorStageSummary private[spark]( val diskBytesSpilled : Long, val isBlacklistedForStage: Boolean) -class ExecutorSummary private[spark]( - val id: String, - val hostPort: String, - val isActive: Boolean, - val rddBlocks: Int, - val memoryUsed: Long, - val diskUsed: Long, - val totalCores: Int, - val maxTasks: Int, - val activeTasks: Int, - val failedTasks: Int, - val completedTasks: Int, - val totalTasks: Int, - val totalDuration: Long, - val totalGCTime: Long, - val totalInputBytes: Long, - val totalShuffleRead: Long, - val totalShuffleWrite: Long, - val isBlacklisted: Boolean, - val maxMemory: Long, - val addTime: Date, - val removeTime: Option[Date], - val removeReason: Option[String], - val executorLogs: Map[String, String], - val memoryMetrics: Option[MemoryMetrics], - val blacklistedInStages: Set[Int], +case class ExecutorSummary private[spark]( + id: String, + hostPort: String, + isActive: Boolean, + rddBlocks: Int, + memoryUsed: Long, + diskUsed: Long, + totalCores: Int, + maxTasks: Int, + activeTasks: Int, + failedTasks: Int, + completedTasks: Int, + totalTasks: Int, + totalDuration: Long, + totalGCTime: Long, + totalInputBytes: Long, + totalShuffleRead: Long, + totalShuffleWrite: Long, + isBlacklisted: Boolean, + maxMemory: Long, + addTime: Date, + removeTime: Option[Date], + removeReason: Option[String], + executorLogs: Map[String, String], + memoryMetrics: Option[MemoryMetrics], + blacklistedInStages: Set[Int], @JsonSerialize(using = classOf[ExecutorMetricsJsonSerializer]) @JsonDeserialize(using = classOf[ExecutorMetricsJsonDeserializer]) - val peakMemoryMetrics: Option[ExecutorMetrics]) + peakMemoryMetrics: Option[ExecutorMetrics], + @JsonIgnore attributes: Map[String, String] = Map.empty) class MemoryMetrics private[spark]( val usedOnHeapStorageMemory: Long, diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index f6f080e87e331..961c3a7e4844e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -423,8 +423,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc app.attempts.foreach { attempt => val appUi = provider.getAppUI(app.id, attempt.attemptId) appUi should not be null - val executors = appUi.get.ui.store.store.view(classOf[ExecutorSummaryWrapper]) - .closeableIterator().asScala + val executors = appUi.get.ui.store.executorList(false).iterator executors should not be null val iterForExpectation = expectedLogUrlMap.iterator @@ -434,8 +433,8 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc val executor = executors.next() val expectation = iterForExpectation.next() - executor.host should be(expectation._1.executorHost) - executor.info.executorLogs should be(expectation._2) + executor.hostPort should startWith(expectation._1.executorHost) + executor.executorLogs should be(expectation._2) executorCount += 1 } From 02117a02a523122416528300c3a1b2b405b8ff57 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Fri, 18 Jan 2019 21:51:10 +0900 Subject: [PATCH 09/28] Fix existing UT break, as well as add unit test on acceptance test for custom url --- .../history/HistoryAppStatusStore.scala | 2 +- .../org/apache/spark/status/api/v1/api.scala | 65 +- .../application_list_json_expectation.json | 16 + .../completed_app_list_json_expectation.json | 16 + ...son_apply_custom_log_urls_expectation.json | 126 ++++ .../executor_list_json_expectation.json | 3 +- ...ith_executor_metrics_json_expectation.json | 24 +- ...process_tree_metrics_json_expectation.json | 6 +- .../executor_memory_usage_expectation.json | 15 +- ...xecutor_node_blacklisting_expectation.json | 15 +- ...acklisting_unblacklisting_expectation.json | 15 +- .../limit_app_list_json_expectation.json | 33 +- .../minDate_app_list_json_expectation.json | 16 + .../minEndDate_app_list_json_expectation.json | 16 + .../application_1547723113049_0005 | 592 ++++++++++++++++++ .../deploy/history/HistoryServerSuite.scala | 63 +- 16 files changed, 939 insertions(+), 84 deletions(-) create mode 100644 core/src/test/resources/HistoryServerExpectations/executor_list_json_apply_custom_log_urls_expectation.json create mode 100644 core/src/test/resources/spark-events/application_1547723113049_0005 diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala index 11c1396a417d5..86e0bfa9cc0c1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala @@ -83,7 +83,7 @@ private[spark] class HistoryAppStatusStore( Map("log" -> updatedUrl) } - exec.copy(executorLogs = newLogUrlMap) + exec.replaceExecutorLogs(newLogUrlMap) } private def logFailToRenewLogUrls( diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala index 8424b9636882c..2a3f501cc0283 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala @@ -78,36 +78,45 @@ class ExecutorStageSummary private[spark]( val diskBytesSpilled : Long, val isBlacklistedForStage: Boolean) -case class ExecutorSummary private[spark]( - id: String, - hostPort: String, - isActive: Boolean, - rddBlocks: Int, - memoryUsed: Long, - diskUsed: Long, - totalCores: Int, - maxTasks: Int, - activeTasks: Int, - failedTasks: Int, - completedTasks: Int, - totalTasks: Int, - totalDuration: Long, - totalGCTime: Long, - totalInputBytes: Long, - totalShuffleRead: Long, - totalShuffleWrite: Long, - isBlacklisted: Boolean, - maxMemory: Long, - addTime: Date, - removeTime: Option[Date], - removeReason: Option[String], - executorLogs: Map[String, String], - memoryMetrics: Option[MemoryMetrics], - blacklistedInStages: Set[Int], +class ExecutorSummary private[spark]( + val id: String, + val hostPort: String, + val isActive: Boolean, + val rddBlocks: Int, + val memoryUsed: Long, + val diskUsed: Long, + val totalCores: Int, + val maxTasks: Int, + val activeTasks: Int, + val failedTasks: Int, + val completedTasks: Int, + val totalTasks: Int, + val totalDuration: Long, + val totalGCTime: Long, + val totalInputBytes: Long, + val totalShuffleRead: Long, + val totalShuffleWrite: Long, + val isBlacklisted: Boolean, + val maxMemory: Long, + val addTime: Date, + val removeTime: Option[Date], + val removeReason: Option[String], + val executorLogs: Map[String, String], + val memoryMetrics: Option[MemoryMetrics], + val blacklistedInStages: Set[Int], @JsonSerialize(using = classOf[ExecutorMetricsJsonSerializer]) @JsonDeserialize(using = classOf[ExecutorMetricsJsonDeserializer]) - peakMemoryMetrics: Option[ExecutorMetrics], - @JsonIgnore attributes: Map[String, String] = Map.empty) + val peakMemoryMetrics: Option[ExecutorMetrics], + @JsonIgnore val attributes: Map[String, String]) { + + def replaceExecutorLogs(newExecutorLogs: Map[String, String]): ExecutorSummary = { + new ExecutorSummary(id, hostPort, isActive, rddBlocks, memoryUsed, diskUsed, totalCores, + maxTasks, activeTasks, failedTasks, completedTasks, totalTasks, totalDuration, + totalGCTime, totalInputBytes, totalShuffleRead, totalShuffleWrite, isBlacklisted, + maxMemory, addTime, removeTime, removeReason, newExecutorLogs, memoryMetrics, + blacklistedInStages, peakMemoryMetrics, attributes) + } +} class MemoryMetrics private[spark]( val usedOnHeapStorageMemory: Long, 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 0f0ccf9858a38..b7b67eea7b298 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,20 @@ [ { + "id" : "application_1547723113049_0005", + "name" : "Spark SQL basic example", + "attempts" : [ { + "attemptId" : "1", + "startTime" : "2019-01-18T05:14:04.248GMT", + "endTime" : "2019-01-18T05:14:39.714GMT", + "lastUpdated" : "", + "duration" : 35466, + "sparkUser" : "spark", + "completed" : true, + "appSparkVersion" : "3.0.0-SNAPSHOT", + "startTimeEpoch" : 1547788444248, + "endTimeEpoch" : 1547788479714, + "lastUpdatedEpoch" : 0 + } ] +}, { "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 e136a35a1e3a9..7d6f1e7eb956f 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,20 @@ [ { + "id" : "application_1547723113049_0005", + "name" : "Spark SQL basic example", + "attempts" : [ { + "attemptId" : "1", + "startTime" : "2019-01-18T05:14:04.248GMT", + "endTime" : "2019-01-18T05:14:39.714GMT", + "lastUpdated" : "", + "duration" : 35466, + "sparkUser" : "spark", + "completed" : true, + "appSparkVersion" : "3.0.0-SNAPSHOT", + "startTimeEpoch" : 1547788444248, + "endTimeEpoch" : 1547788479714, + "lastUpdatedEpoch" : 0 + } ] +}, { "id" : "application_1538416563558_0014", "name" : "PythonBisectingKMeansExample", "attempts" : [ { diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_json_apply_custom_log_urls_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_json_apply_custom_log_urls_expectation.json new file mode 100644 index 0000000000000..61ba87e4c656d --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/executor_list_json_apply_custom_log_urls_expectation.json @@ -0,0 +1,126 @@ +[ + { + "id":"driver", + "hostPort":"node2:40675", + "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":428762726, + "addTime":"2019-01-18T05:14:05.481GMT", + "executorLogs":{ + "stdout":"http://node2:8042/node/containerlogs/container_e01_1547723113049_0005_01_000001/spark/stdout?start=-4096", + "stderr":"http://node2:8042/node/containerlogs/container_e01_1547723113049_0005_01_000001/spark/stderr?start=-4096" + }, + "memoryMetrics":{ + "usedOnHeapStorageMemory":0, + "usedOffHeapStorageMemory":0, + "totalOnHeapStorageMemory":428762726, + "totalOffHeapStorageMemory":0 + }, + "blacklistedInStages":[ + + ], + "attributes":{ + + } + }, + { + "id":"2", + "hostPort":"node4:44875", + "isActive":true, + "rddBlocks":0, + "memoryUsed":0, + "diskUsed":0, + "totalCores":1, + "maxTasks":1, + "activeTasks":0, + "failedTasks":0, + "completedTasks":70, + "totalTasks":70, + "totalDuration":6250, + "totalGCTime":162, + "totalInputBytes":0, + "totalShuffleRead":0, + "totalShuffleWrite":0, + "isBlacklisted":false, + "maxMemory":384093388, + "addTime":"2019-01-18T05:14:15.343GMT", + "executorLogs":{ + "stderr":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000003/stderr)", + "stdout":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000003/stdout)" + }, + "memoryMetrics":{ + "usedOnHeapStorageMemory":0, + "usedOffHeapStorageMemory":0, + "totalOnHeapStorageMemory":384093388, + "totalOffHeapStorageMemory":0 + }, + "blacklistedInStages":[ + + ], + "attributes":{ + "NODE_HTTP_ADDRESS":"node4:8042", + "USER":"spark", + "LOG_FILES":"stderr,stdout", + "CLUSTER_ID":"yarn-cluster", + "HTTP_SCHEME":"http://", + "CONTAINER_ID":"container_e01_1547723113049_0005_01_000003" + } + }, + { + "id":"1", + "hostPort":"node3:35087", + "isActive":true, + "rddBlocks":0, + "memoryUsed":0, + "diskUsed":0, + "totalCores":1, + "maxTasks":1, + "activeTasks":0, + "failedTasks":0, + "completedTasks":151, + "totalTasks":151, + "totalDuration":13343, + "totalGCTime":264, + "totalInputBytes":995, + "totalShuffleRead":186, + "totalShuffleWrite":186, + "isBlacklisted":false, + "maxMemory":384093388, + "addTime":"2019-01-18T05:14:14.209GMT", + "executorLogs":{ + "stderr":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000002/stderr)", + "stdout":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000002/stdout)" + }, + "memoryMetrics":{ + "usedOnHeapStorageMemory":0, + "usedOffHeapStorageMemory":0, + "totalOnHeapStorageMemory":384093388, + "totalOffHeapStorageMemory":0 + }, + "blacklistedInStages":[ + + ], + "attributes":{ + "NODE_HTTP_ADDRESS":"node3:8042", + "USER":"spark", + "LOG_FILES":"stderr,stdout", + "CLUSTER_ID":"yarn-cluster", + "HTTP_SCHEME":"http://", + "CONTAINER_ID":"container_e01_1547723113049_0005_01_000002" + } + } +] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json index 7bb8fe8fd8f98..209164e3f137d 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json @@ -20,5 +20,6 @@ "maxMemory" : 278302556, "addTime" : "2015-02-03T16:43:00.906GMT", "executorLogs" : { }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } } ] 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 75674778dd1f6..f282d483a4fba 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 @@ -44,7 +44,8 @@ "ProcessTreePythonRSSMemory": 0, "ProcessTreeOtherVMemory": 0, "ProcessTreeOtherRSSMemory": 0 - } + }, + "attributes" : { } }, { "id" : "7", "hostPort" : "node6340.grid.company.com:5933", @@ -76,7 +77,8 @@ "totalOnHeapStorageMemory" : 956615884, "totalOffHeapStorageMemory" : 0 }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } }, { "id" : "6", "hostPort" : "node6644.grid.company.com:8445", @@ -108,7 +110,8 @@ "totalOnHeapStorageMemory" : 956615884, "totalOffHeapStorageMemory" : 0 }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } }, { "id" : "5", "hostPort" : "node2477.grid.company.com:20123", @@ -140,7 +143,8 @@ "totalOnHeapStorageMemory" : 956615884, "totalOffHeapStorageMemory" : 0 }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } }, { "id" : "4", "hostPort" : "node4243.grid.company.com:16084", @@ -190,7 +194,8 @@ "ProcessTreePythonRSSMemory": 0, "ProcessTreeOtherVMemory": 0, "ProcessTreeOtherRSSMemory": 0 - } + }, + "attributes" : { } }, { "id" : "3", "hostPort" : "node0998.grid.company.com:45265", @@ -240,7 +245,8 @@ "ProcessTreePythonRSSMemory": 0, "ProcessTreeOtherVMemory": 0, "ProcessTreeOtherRSSMemory": 0 - } + }, + "attributes" : { } }, { "id" : "2", "hostPort" : "node4045.grid.company.com:29262", @@ -290,7 +296,8 @@ "ProcessTreePythonRSSMemory": 0, "ProcessTreeOtherVMemory": 0, "ProcessTreeOtherRSSMemory": 0 - } + }, + "attributes" : { } }, { "id" : "1", "hostPort" : "node1404.grid.company.com:34043", @@ -340,5 +347,6 @@ "ProcessTreePythonRSSMemory": 0, "ProcessTreeOtherVMemory": 0, "ProcessTreeOtherRSSMemory": 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 69efefe736dd4..980ec85aadad0 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 @@ -44,7 +44,8 @@ "ProcessTreePythonRSSMemory" : 40284160, "ProcessTreeOtherVMemory" : 0, "ProcessTreeOtherRSSMemory" : 0 - } + }, + "attributes" : { } }, { "id" : "9", "hostPort" : "rezamemory-2.gce.something.com:40797", @@ -94,5 +95,6 @@ "ProcessTreePythonRSSMemory" : 69013504, "ProcessTreeOtherVMemory" : 0, "ProcessTreeOtherRSSMemory" : 0 - } + }, + "attributes" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json index dd5b1dcb7372b..c7f0a86211ad7 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json @@ -26,7 +26,8 @@ "totalOnHeapStorageMemory" : 384093388, "totalOffHeapStorageMemory" : 524288000 }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } }, { "id" : "3", "hostPort" : "172.22.0.167:51485", @@ -58,7 +59,8 @@ "totalOnHeapStorageMemory" : 384093388, "totalOffHeapStorageMemory" : 524288000 }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } } ,{ "id" : "2", "hostPort" : "172.22.0.167:51487", @@ -90,7 +92,8 @@ "totalOnHeapStorageMemory" : 384093388, "totalOffHeapStorageMemory" : 524288000 }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } }, { "id" : "1", "hostPort" : "172.22.0.167:51490", @@ -122,7 +125,8 @@ "totalOnHeapStorageMemory": 384093388, "totalOffHeapStorageMemory": 524288000 }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } }, { "id" : "0", "hostPort" : "172.22.0.167:51491", @@ -154,5 +158,6 @@ "totalOnHeapStorageMemory" : 384093388, "totalOffHeapStorageMemory" : 524288000 }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json index 3e55d3d9d7eb9..02a0ecd7f4d80 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json @@ -26,7 +26,8 @@ "totalOnHeapStorageMemory" : 384093388, "totalOffHeapStorageMemory" : 524288000 }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } }, { "id" : "3", "hostPort" : "172.22.0.167:51485", @@ -58,7 +59,8 @@ "totalOnHeapStorageMemory" : 384093388, "totalOffHeapStorageMemory" : 524288000 }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } }, { "id" : "2", "hostPort" : "172.22.0.167:51487", @@ -90,7 +92,8 @@ "totalOnHeapStorageMemory" : 384093388, "totalOffHeapStorageMemory" : 524288000 }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } }, { "id" : "1", "hostPort" : "172.22.0.167:51490", @@ -122,7 +125,8 @@ "totalOnHeapStorageMemory": 384093388, "totalOffHeapStorageMemory": 524288000 }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } }, { "id" : "0", "hostPort" : "172.22.0.167:51491", @@ -154,5 +158,6 @@ "totalOnHeapStorageMemory": 384093388, "totalOffHeapStorageMemory": 524288000 }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json index e87f3e78f2dc8..4d31a387fbc53 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json @@ -20,7 +20,8 @@ "maxMemory" : 384093388, "addTime" : "2016-11-15T23:20:38.836GMT", "executorLogs" : { }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } }, { "id" : "3", "hostPort" : "172.22.0.111:64543", @@ -46,7 +47,8 @@ "stdout" : "http://172.22.0.111:64521/logPage/?appId=app-20161115172038-0000&executorId=3&logType=stdout", "stderr" : "http://172.22.0.111:64521/logPage/?appId=app-20161115172038-0000&executorId=3&logType=stderr" }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } }, { "id" : "2", "hostPort" : "172.22.0.111:64539", @@ -72,7 +74,8 @@ "stdout" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stdout", "stderr" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stderr" }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } }, { "id" : "1", "hostPort" : "172.22.0.111:64541", @@ -98,7 +101,8 @@ "stdout" : "http://172.22.0.111:64518/logPage/?appId=app-20161115172038-0000&executorId=1&logType=stdout", "stderr" : "http://172.22.0.111:64518/logPage/?appId=app-20161115172038-0000&executorId=1&logType=stderr" }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } }, { "id" : "0", "hostPort" : "172.22.0.111:64540", @@ -124,5 +128,6 @@ "stdout" : "http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stdout", "stderr" : "http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stderr" }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "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 0ef9377dcb08b..0305df8c7c33b 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,20 @@ [ { + "id" : "application_1547723113049_0005", + "name" : "Spark SQL basic example", + "attempts" : [ { + "attemptId" : "1", + "startTime" : "2019-01-18T05:14:04.248GMT", + "endTime" : "2019-01-18T05:14:39.714GMT", + "lastUpdated" : "", + "duration" : 35466, + "sparkUser" : "spark", + "completed" : true, + "appSparkVersion" : "3.0.0-SNAPSHOT", + "startTimeEpoch" : 1547788444248, + "endTimeEpoch" : 1547788479714, + "lastUpdatedEpoch" : 0 + } ] +}, { "id" : "application_1538416563558_0014", "name" : "PythonBisectingKMeansExample", "attempts" : [ { @@ -28,19 +44,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 ea9dc1b97afc8..681a8bc781260 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,20 @@ [ { + "id" : "application_1547723113049_0005", + "name" : "Spark SQL basic example", + "attempts" : [ { + "attemptId" : "1", + "startTime" : "2019-01-18T05:14:04.248GMT", + "endTime" : "2019-01-18T05:14:39.714GMT", + "lastUpdated" : "", + "duration" : 35466, + "sparkUser" : "spark", + "completed" : true, + "appSparkVersion" : "3.0.0-SNAPSHOT", + "startTimeEpoch" : 1547788444248, + "endTimeEpoch" : 1547788479714, + "lastUpdatedEpoch" : 0 + } ] +}, { "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 2a77071a9ffd9..a50b74ee50e55 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,20 @@ [ { + "id" : "application_1547723113049_0005", + "name" : "Spark SQL basic example", + "attempts" : [ { + "attemptId" : "1", + "startTime" : "2019-01-18T05:14:04.248GMT", + "endTime" : "2019-01-18T05:14:39.714GMT", + "lastUpdated" : "", + "duration" : 35466, + "sparkUser" : "spark", + "completed" : true, + "appSparkVersion" : "3.0.0-SNAPSHOT", + "startTimeEpoch" : 1547788444248, + "endTimeEpoch" : 1547788479714, + "lastUpdatedEpoch" : 0 + } ] +}, { "id" : "application_1538416563558_0014", "name" : "PythonBisectingKMeansExample", "attempts" : [ { diff --git a/core/src/test/resources/spark-events/application_1547723113049_0005 b/core/src/test/resources/spark-events/application_1547723113049_0005 new file mode 100644 index 0000000000000..718a7e6e0fe00 --- /dev/null +++ b/core/src/test/resources/spark-events/application_1547723113049_0005 @@ -0,0 +1,592 @@ +{"Event":"SparkListenerLogStart","Spark Version":"3.0.0-SNAPSHOT"} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"node2","Port":40675},"Maximum Memory":428762726,"Timestamp":1547788445481,"Maximum Onheap Memory":428762726,"Maximum Offheap Memory":0} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre","Java Version":"1.8.0_161 (Oracle Corporation)","Scala Version":"version 2.12.8"},"Spark Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.scheduler.mode":"FIFO","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"},"System Properties":{"java.io.tmpdir":"/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_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/yarn","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/amd64","user.dir":"/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","java.library.path":"/usr/java/packages/lib/amd64:/usr/lib64:/lib64:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.161-b14","jetty.git.hash":"27208684755d94a92186989f695db2d7b21ebc51","java.endorsed.dirs":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/endorsed","java.runtime.version":"1.8.0_161-b14","java.vm.info":"mixed mode","java.ext.dirs":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"OpenJDK Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/resources.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/rt.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/sunrsasign.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/jsse.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/jce.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/charsets.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/jfr.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/classes","file.encoding":"UTF-8","user.timezone":"UTC","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"4.18.5-1.el7.elrepo.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":"yarn","hdp.version":"2.6.5.1002-73","java.vm.name":"OpenJDK 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.yarn.ApplicationMaster --class org.apache.spark.examples.sql.SparkSQLExample --jar file:/home/spark/spark-3.0.0-snapshot-spark-26311-v2/examples/jars/spark-examples_2.12-3.0.0-SNAPSHOT.jar --properties-file /hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_conf__/__spark_conf__.properties --dist-cache-conf /hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_conf__/__spark_dist_cache__.properties","java.home":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre","java.version":"1.8.0_161","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/hadoop-yarn-client/lib/commons-digester-1.8.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-applications-distributedshell-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-compiler-3.0.11.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-hadoop-bundle-1.6.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/lz4-java-1.5.0.jar":"System Classpath","/hadoop/lib/commons-digester-1.8.jar":"System Classpath","/hadoop-yarn-client/lib/netty-3.6.2.Final.jar":"System Classpath","/hadoop/lib/curator-client-2.7.1.jar":"System Classpath","/hadoop/lib/log4j-1.2.17.jar":"System Classpath","/hadoop/lib/paranamer-2.3.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-codec-1.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hk2-api-2.4.0-b34.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/orc-core-1.5.4-nohive.jar":"System Classpath","/hadoop-yarn-client/lib/curator-client-2.7.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop/hadoop-common.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-cli-1.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-hdfs-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/aopalliance-1.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-guava-2.22.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/stax-api-1.0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/snappy-java-1.1.7.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/univocity-parsers-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/azure-storage-5.4.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/netty-all-4.1.30.Final.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-container-servlet-core-2.22.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/macro-compat_2.12-1.1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-httpclient-3.1.jar":"System Classpath","/hadoop/lib/httpcore-4.4.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/hadoop/lib/java-xmlbuilder-0.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jodd-core-3.5.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/breeze-macros_2.12-0.13.2.jar":"System Classpath","/hadoop-yarn-client/lib/zookeeper-3.4.6.jar":"System Classpath","/hadoop/lib/curator-framework-2.7.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jul-to-slf4j-1.7.16.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/xz-1.5.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/antlr-2.7.7.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/json4s-scalap_2.12-3.5.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/apache-log4j-extras-1.2.17.jar":"System Classpath","/hadoop-hdfs-client/lib/htrace-core-3.1.0-incubating.jar":"System Classpath","/hadoop/hadoop-azure.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-nodemanager.jar":"System Classpath","/hadoop-yarn-client/lib/servlet-api-2.5.jar":"System Classpath","/hadoop-yarn-client/lib/guava-11.0.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/arpack_combined_all-0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/arrow-format-0.10.0.jar":"System Classpath","/hadoop-yarn-client/lib/guice-3.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/scala-library-2.12.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-container-servlet-2.22.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/xercesImpl-2.9.1.jar":"System Classpath","/hadoop/hadoop-azure-datalake-2.7.3.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-common.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/yarn/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-auth-2.7.4.jar":"System Classpath","/hadoop/lib/activation-1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jline-2.14.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/yarn/lib/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hive-metastore-1.2.1.spark2.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-applicationhistoryservice.jar":"System Classpath","/hadoop/lib/ojdbc6.jar":"System Classpath","/hadoop/lib/commons-collections-3.2.2.jar":"System Classpath","/hadoop-hdfs-client/lib/servlet-api-2.5.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/joda-time-2.9.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-annotations-2.9.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-mllib-local_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/leveldbjni-all-1.8.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-common-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/json4s-jackson_2.12-3.5.3.jar":"System Classpath","/hadoop/conf":"System Classpath","/hadoop/hadoop-common-2.7.3-tests.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-timeline-pluginstorage-2.7.3.jar":"System Classpath","/hadoop/lib/junit-4.11.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs-2.7.3-tests.jar":"System Classpath","/hadoop-yarn-client/lib/httpcore-4.4.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-unsafe_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/metrics-jvm-3.1.5.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/objenesis-2.5.1.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-nodemanager-2.7.3.jar":"System Classpath","/hadoop/hadoop-common-tests.jar":"System Classpath","/hadoop-hdfs-client/lib/jackson-databind-2.2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-graphx_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-hdfs-client/lib/asm-3.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/kryo-shaded-4.0.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/compress-lzf-1.0.3.jar":"System Classpath","/hadoop/lib/azure-storage-5.4.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/avro-mapred-1.8.2-hadoop2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/scala-xml_2.12-1.0.5.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-applications-distributedshell.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jaxb-api-2.2.2.jar":"System Classpath","/hadoop/lib/commons-net-3.1.jar":"System Classpath","/hadoop-yarn-client/lib/java-xmlbuilder-0.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/paranamer-2.8.jar":"System Classpath","/hadoop/lib/commons-logging-1.1.3.jar":"System Classpath","/hadoop-yarn-client/lib/log4j-1.2.17.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/mapreduce/*":"System Classpath","/hadoop/lib/commons-codec-1.4.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs-nfs-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/commons-lang3-3.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-math3-3.4.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-network-common_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jetty-6.1.26.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/javolution-5.5.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jcl-over-slf4j-1.7.16.jar":"System Classpath","/hadoop/lib/slf4j-api-1.7.10.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-common-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-core-asl-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hk2-utils-2.4.0-b34.jar":"System Classpath","/hadoop/lib/mockito-all-1.8.5.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-client.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/common/lib/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hppc-0.7.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/shapeless_2.12-2.3.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-mapreduce-client-common-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/log4j-1.2.17.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-module-scala_2.12-2.9.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-module-paranamer-2.9.6.jar":"System Classpath","/hadoop-hdfs-client/lib/jetty-6.1.26.hwx.jar":"System Classpath","/hadoop/hadoop-aws.jar":"System Classpath","/hadoop/lib/commons-cli-1.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-codec-1.10.jar":"System Classpath","/hadoop/lib/commons-compress-1.4.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-server-2.22.2.jar":"System Classpath","/hadoop/lib/avro-1.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-xc-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-dbcp-1.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-beanutils-1.7.0.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-jaxrs-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/api-asn1-api-1.0.0-M20.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-client-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/javax.inject-2.4.0-b34.jar":"System Classpath","/hadoop/gcs-connector-1.8.1-shaded.jar":"System Classpath","/hadoop-hdfs-client/lib/jersey-core-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/metrics-json-3.1.5.jar":"System Classpath","/hadoop/lib/jackson-core-2.2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jpam-1.1.jar":"System Classpath","/hadoop/lib/jersey-core-1.9.jar":"System Classpath","/hadoop/lib/snappy-java-1.0.4.1.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-lang-2.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/slf4j-log4j12-1.7.16.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/osgi-resource-locator-1.0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/api-asn1-api-1.0.0-M20.jar":"System Classpath","/hadoop-yarn-client/lib/nimbus-jose-jwt-4.41.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-databind-2.9.6.jar":"System Classpath","/hadoop-hdfs-client/lib/xml-apis-1.3.04.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-client-2.22.2.jar":"System Classpath","/hadoop/hadoop-nfs-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hive-jdbc-1.2.1.spark2.jar":"System Classpath","/hadoop/lib/jackson-databind-2.2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/eigenbase-properties-1.1.5.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/machinist_2.12-0.6.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/tools/lib/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-jackson-1.10.0.jar":"System Classpath","/hadoop-yarn-client/lib/jetty-util-6.1.26.hwx.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-digester-1.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/calcite-avatica-1.2.0-incubating.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/oro-2.0.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hive-cli-1.2.1.spark2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-hive-thriftserver_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-applications-unmanaged-am-launcher-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/JavaEWAH-0.3.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/super-csv-2.2.0.jar":"System Classpath","/hadoop/lib/ranger-yarn-plugin-shim-0.7.0.jar":"System Classpath","/hadoop-yarn-client/lib/jetty-6.1.26.hwx.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/metrics-core-3.1.5.jar":"System Classpath","/hadoop-yarn-client/lib/commons-compress-1.4.1.jar":"System Classpath","/hadoop-hdfs-client/lib/netty-all-4.0.52.Final.jar":"System Classpath","/hadoop/hadoop-auth.jar":"System Classpath","/hadoop-yarn-client/lib/javax.inject-1.jar":"System Classpath","/hadoop/lib/jettison-1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jetty-util-6.1.26.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-network-shuffle_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/lib/jackson-jaxrs-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/json-smart-1.3.1.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-api-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/commons-beanutils-1.7.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-streaming_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/lib/commons-io-2.4.jar":"System Classpath","/hadoop/lib/commons-beanutils-1.7.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/snappy-0.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jta-1.1.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-resourcemanager-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/protobuf-java-2.5.0.jar":"System Classpath","/hadoop-yarn-client/lib/curator-framework-2.7.1.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-daemon-1.0.13.jar":"System Classpath","/hadoop/lib/hadoop-lzo-0.6.0-sources.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-common.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/breeze_2.12-0.13.2.jar":"System Classpath","/hadoop/lib/jsp-api-2.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/orc-mapreduce-1.5.4-nohive.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-mapreduce-client-core-2.7.4.jar":"System Classpath","/hadoop/lib/htrace-core-3.1.0-incubating.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-encoding-1.10.0.jar":"System Classpath","/hadoop/hadoop-auth-2.7.3.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-applicationhistoryservice-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/jettison-1.1.jar":"System Classpath","/hadoop-yarn-client/lib/commons-codec-1.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-mapreduce-client-shuffle-2.7.4.jar":"System Classpath","/hadoop/lib/curator-recipes-2.7.1.jar":"System Classpath","/hadoop/lib/jetty-util-6.1.26.hwx.jar":"System Classpath","/hadoop-hdfs-client/lib/log4j-1.2.17.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/activation-1.1.1.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/jaxb-api-2.2.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/opencsv-2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-collections-3.2.2.jar":"System Classpath","/hadoop/lib/guava-11.0.2.jar":"System Classpath","/hadoop-yarn-client/lib/commons-net-3.1.jar":"System Classpath","/hadoop/lib/commons-beanutils-core-1.8.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/py4j-0.10.8.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/json4s-ast_2.12-3.5.3.jar":"System Classpath","/hadoop-yarn-client/lib/metrics-core-3.0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/pyrolite-4.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/chill_2.12-0.9.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/libfb303-0.9.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/javax.inject-1.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs.jar":"System Classpath","/ext/hadoop/*":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-web-proxy-2.7.3.jar":"System Classpath","/hadoop/lib/aws-java-sdk-core-1.10.6.jar":"System Classpath","/hadoop-yarn-client/lib/commons-configuration-1.6.jar":"System Classpath","/hadoop/lib/zookeeper-3.4.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-logging-1.1.3.jar":"System Classpath","/hadoop-hdfs-client/lib/jsr305-3.0.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/stringtemplate-3.2.1.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-resourcemanager.jar":"System Classpath","/hadoop/lib/netty-3.6.2.Final.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/zstd-jni-1.3.2-2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jsp-api-2.1.jar":"System Classpath","/hadoop-yarn-client/lib/jaxb-impl-2.2.3-1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-tags_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hive-beeline-1.2.1.spark2.jar":"System Classpath","/hadoop-hdfs-client/lib/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/snappy-java-1.0.4.1.jar":"System Classpath","/hadoop-yarn-client/lib/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs-nfs.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/hadoop/hadoop-aws-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/jsr305-3.0.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/datanucleus-rdbms-3.2.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/libthrift-0.9.3.jar":"System Classpath","/hadoop/lib/xz-1.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/json4s-core_2.12-3.5.3.jar":"System Classpath","/hadoop-yarn-client/lib/jersey-guice-1.9.jar":"System Classpath","/hadoop-yarn-client/lib/activation-1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-jaxrs-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/hdfs/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/mapreduce/lib/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-pool-1.5.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/arrow-memory-0.10.0.jar":"System Classpath","/hadoop-hdfs-client/lib/jetty-util-6.1.26.hwx.jar":"System Classpath","/hadoop/lib/ranger-plugin-classloader-0.7.0.jar":"System Classpath","/hadoop/lib/aws-java-sdk-kms-1.10.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-repl_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-yarn-client/lib/jersey-json-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-yarn_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-yarn-client/lib/xz-1.0.jar":"System Classpath","/hadoop/hadoop-common-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jetty-sslengine-6.1.26.jar":"System Classpath","/hadoop/lib/jaxb-api-2.2.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/metrics-graphite-3.1.5.jar":"System Classpath","/hadoop/lib/jcip-annotations-1.0-1.jar":"System Classpath","/hadoop/lib/nimbus-jose-jwt-4.41.1.jar":"System Classpath","/hadoop/lib/hadoop-lzo-0.6.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hk2-locator-2.4.0-b34.jar":"System Classpath","/hadoop/lib/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-logging-1.1.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jsr305-3.0.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-hive_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/RoaringBitmap-0.5.11.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/calcite-linq4j-1.2.0-incubating.jar":"System Classpath","/hadoop-yarn-client/lib/commons-beanutils-core-1.8.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-lang-2.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/derby-10.12.1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-hadoop-1.10.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/stax-api-1.0-2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/scala-reflect-2.12.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/calcite-core-1.2.0-incubating.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_conf__":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-format-2.4.0.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-sharedcachemanager-2.7.3.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-web-proxy.jar":"System Classpath","/hadoop-hdfs-client/lib/xercesImpl-2.9.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/netty-3.9.9.Final.jar":"System Classpath","/hadoop/lib/xmlenc-0.52.jar":"System Classpath","/hadoop/hadoop-annotations-2.7.3.jar":"System Classpath","/hadoop-hdfs-client/lib/protobuf-java-2.5.0.jar":"System Classpath","/hadoop-yarn-client/lib/jersey-core-1.9.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-registry-2.7.3.jar":"System Classpath","/hadoop-hdfs-client/lib/okhttp-2.7.5.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-yarn-common-2.7.4.jar":"System Classpath","/hadoop-yarn-client/lib/leveldbjni-all-1.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-crypto-1.0.0.jar":"System Classpath","/hadoop/lib/jetty-6.1.26.hwx.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/aopalliance-repackaged-2.4.0-b34.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_conf__/__hadoop_conf__":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-compress-1.8.1.jar":"System Classpath","/hadoop/lib/jackson-xc-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/gson-2.2.4.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-api.jar":"System Classpath","/hadoop/lib/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs-tests.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-io-2.4.jar":"System Classpath","/hadoop/hadoop-nfs.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-yarn-api-2.7.4.jar":"System Classpath","/hadoop/lib/json-smart-1.3.1.jar":"System Classpath","/hadoop-yarn-client/lib/commons-lang-2.6.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/azure-keyvault-core-0.8.0.jar":"System Classpath","/hadoop-yarn-client/lib/fst-2.24.jar":"System Classpath","/hadoop-yarn-client/lib/htrace-core-3.1.0-incubating.jar":"System Classpath","/hadoop/hadoop-azure-datalake.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-common-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hive-exec-1.2.1.spark2.jar":"System Classpath","/hadoop-hdfs-client/lib/jersey-server-1.9.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-sharedcachemanager.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-launcher_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/datanucleus-core-3.2.10.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/zookeeper-3.4.6.jar":"System Classpath","/hadoop-yarn-client/lib/commons-math3-3.1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/api-util-1.0.0-M20.jar":"System Classpath","/hadoop/lib/azure-keyvault-core-0.8.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-common-2.22.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jtransforms-2.4.0.jar":"System Classpath","/hadoop/lib/asm-3.2.jar":"System Classpath","/hadoop/lib/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/javax.ws.rs-api-2.0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/xmlenc-0.52.jar":"System Classpath","/hadoop/lib/jsch-0.1.54.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-timeline-pluginstorage.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-annotations-2.2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/hdfs/lib/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/httpclient-4.5.6.jar":"System Classpath","/hadoop-hdfs-client/lib/xmlenc-0.52.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-annotations-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/antlr-runtime-3.4.jar":"System Classpath","/etc/hadoop/conf/secure":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/avro-ipc-1.8.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/janino-3.0.11.jar":"System Classpath","/hadoop/lib/api-util-1.0.0-M20.jar":"System Classpath","/hadoop-yarn-client/lib/xmlenc-0.52.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/slf4j-api-1.7.16.jar":"System Classpath","/hadoop/lib/ranger-hdfs-plugin-shim-0.7.0.jar":"System Classpath","/hadoop-hdfs-client/lib/guava-11.0.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/chill-java-0.9.3.jar":"System Classpath","/hadoop-yarn-client/lib/aopalliance-1.0.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-applications-unmanaged-am-launcher.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/htrace-core-3.1.0-incubating.jar":"System Classpath","/hadoop/lib/jersey-json-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/guava-14.0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-yarn-server-web-proxy-2.7.4.jar":"System Classpath","/hadoop/lib/jetty-sslengine-6.1.26.hwx.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/ST4-4.0.4.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-client-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-sketch_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-yarn-client/lib/httpclient-4.5.2.jar":"System Classpath","/hadoop/lib/httpclient-4.5.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/antlr4-runtime-4.7.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/arrow-vector-0.10.0.jar":"System Classpath","/hadoop-yarn-client/lib/jersey-server-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-yarn-client-2.7.4.jar":"System Classpath","/hadoop-yarn-client/lib/jsch-0.1.54.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/curator-client-2.7.1.jar":"System Classpath","/hadoop-yarn-client/lib/stax-api-1.0-2.jar":"System Classpath","/hadoop-yarn-client/lib/commons-cli-1.2.jar":"System Classpath","/hadoop/lib/servlet-api-2.5.jar":"System Classpath","/hadoop-yarn-client/lib/commons-collections-3.2.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-net-3.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-media-jaxb-2.22.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-mapreduce-client-app-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-core_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-catalyst_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-kvstore_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/lib/joda-time-2.9.4.jar":"System Classpath","/hadoop-yarn-client/lib/jersey-client-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-lang3-3.8.1.jar":"System Classpath","/hadoop/lib/slf4j-log4j12-1.7.10.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/bonecp-0.8.0.RELEASE.jar":"System Classpath","/hadoop-yarn-client/lib/jetty-sslengine-6.1.26.hwx.jar":"System Classpath","/hadoop-yarn-client/lib/commons-logging-1.1.3.jar":"System Classpath","/hadoop/lib/jets3t-0.9.0.jar":"System Classpath","/hadoop/lib/hadoop-lzo-0.6.0-javadoc.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/orc-shims-1.5.4.jar":"System Classpath","/hadoop-hdfs-client/lib/okio-1.6.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-column-1.10.0.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-core-asl-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/jsp-api-2.1.jar":"System Classpath","/hadoop/lib/aws-java-sdk-s3-1.10.6.jar":"System Classpath","/hadoop/azure-data-lake-store-sdk-2.2.5.jar":"System Classpath","/hadoop-yarn-client/lib/zookeeper-3.4.6-tests.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/stream-2.7.0.jar":"System Classpath","/hadoop/hadoop-azure-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/jcip-annotations-1.0-1.jar":"System Classpath","/hadoop-yarn-client/lib/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/common/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/aircompressor-0.10.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/httpcore-4.4.10.jar":"System Classpath","/hadoop-yarn-client/lib/objenesis-2.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/avro-1.8.2.jar":"System Classpath","/hadoop-yarn-client/lib/jets3t-0.9.0.jar":"System Classpath","/hadoop-hdfs-client/lib/jackson-core-2.2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-configuration-1.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-mapreduce-client-jobclient-2.7.4.jar":"System Classpath","/hadoop-yarn-client/lib/commons-io-2.4.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-tests.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/guice-3.0.jar":"System Classpath","/hadoop/lib/jsr305-3.0.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/javassist-3.18.1-GA.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/curator-framework-2.7.1.jar":"System Classpath","/hadoop-yarn-client/lib/avro-1.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/scala-compiler-2.12.8.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-registry.jar":"System Classpath","/hadoop/lib/stax-api-1.0-2.jar":"System Classpath","/hadoop-yarn-client/lib/guice-servlet-3.0.jar":"System Classpath","/hadoop/lib/commons-math3-3.1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/javax.annotation-api-1.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jdo-api-3.0.1.jar":"System Classpath","/hadoop-hdfs-client/lib/jackson-core-asl-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-sql_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-yarn-server-common-2.7.4.jar":"System Classpath","/hadoop-yarn-client/lib/api-util-1.0.0-M20.jar":"System Classpath","/hadoop/lib/api-asn1-api-1.0.0-M20.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/minlog-1.3.0.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-xc-1.9.13.jar":"System Classpath","/hadoop-hdfs-client/lib/netty-3.6.2.Final.jar":"System Classpath","/hadoop/lib/hamcrest-core-1.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/xbean-asm7-shaded-4.12.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-core-2.2.3.jar":"System Classpath","/hadoop/lib/jaxb-impl-2.2.3-1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/scala-parser-combinators_2.12-1.1.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-io-2.4.jar":"System Classpath","/hadoop/lib/commons-lang-2.6.jar":"System Classpath","/hadoop/lib/commons-lang3-3.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spire_2.12-0.13.0.jar":"System Classpath","/hadoop-hdfs-client/lib/jackson-annotations-2.2.3.jar":"System Classpath","/hadoop/lib/jackson-core-asl-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/javax.servlet-api-3.1.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/core-1.1.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-cli-1.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/curator-recipes-2.7.1.jar":"System Classpath","/hadoop/lib/jersey-server-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spire-macros_2.12-0.13.0.jar":"System Classpath","/hadoop-yarn-client/lib/javassist-3.18.1-GA.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/validation-api-1.1.0.Final.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/protobuf-java-2.5.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-beanutils-core-1.8.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/gson-2.2.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/ivy-2.4.0.jar":"System Classpath","/hadoop-yarn-client/lib/paranamer-2.3.jar":"System Classpath","/hadoop/lib/gson-2.2.4.jar":"System Classpath","/hadoop/lib/jackson-annotations-2.2.3.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-databind-2.2.3.jar":"System Classpath","/hadoop-hdfs-client/lib/leveldbjni-all-1.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/guice-servlet-3.0.jar":"System Classpath","/hadoop/hadoop-annotations.jar":"System Classpath","/hadoop-yarn-client/lib/curator-recipes-2.7.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/flatbuffers-1.2.0-3f79e055.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-mllib_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-tests-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-common-1.10.0.jar":"System Classpath","/hadoop-yarn-client/lib/asm-3.2.jar":"System Classpath","/hadoop/lib/protobuf-java-2.5.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-core-2.9.6.jar":"System Classpath","/hadoop/lib/commons-configuration-1.6.jar":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Spark SQL basic example","App ID":"application_1547723113049_0005","Timestamp":1547788444248,"User":"spark","App Attempt ID":"1","Driver Logs":{"stdout":"http://node2:8042/node/containerlogs/container_e01_1547723113049_0005_01_000001/spark/stdout?start=-4096","stderr":"http://node2:8042/node/containerlogs/container_e01_1547723113049_0005_01_000001/spark/stderr?start=-4096"}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1547788454209,"Executor ID":"1","Executor Info":{"Host":"node3","Total Cores":1,"Log Urls":{"stdout":"http://node3:8042/node/containerlogs/container_e01_1547723113049_0005_01_000002/spark/stdout?start=-4096","stderr":"http://node3:8042/node/containerlogs/container_e01_1547723113049_0005_01_000002/spark/stderr?start=-4096"},"Attributes":{"NODE_HTTP_ADDRESS":"node3:8042","USER":"spark","LOG_FILES":"stderr,stdout","CLUSTER_ID":"yarn-cluster","HTTP_SCHEME":"http://","CONTAINER_ID":"container_e01_1547723113049_0005_01_000002"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"node3","Port":35087},"Maximum Memory":384093388,"Timestamp":1547788454413,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"Event":"SparkListenerExecutorAdded","Timestamp":1547788455343,"Executor ID":"2","Executor Info":{"Host":"node4","Total Cores":1,"Log Urls":{"stdout":"http://node4:8042/node/containerlogs/container_e01_1547723113049_0005_01_000003/spark/stdout?start=-4096","stderr":"http://node4:8042/node/containerlogs/container_e01_1547723113049_0005_01_000003/spark/stderr?start=-4096"},"Attributes":{"NODE_HTTP_ADDRESS":"node4:8042","USER":"spark","LOG_FILES":"stderr,stdout","CLUSTER_ID":"yarn-cluster","HTTP_SCHEME":"http://","CONTAINER_ID":"container_e01_1547723113049_0005_01_000003"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"2","Host":"node4","Port":44875},"Maximum Memory":384093388,"Timestamp":1547788455624,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1547788459293,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"json at SparkSQLExample.scala:59","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"mapPartitions\"}","Callsite":"json at SparkSQLExample.scala:59","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"json at SparkSQLExample.scala:59","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"json at SparkSQLExample.scala:59","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameReader.json(DataFrameReader.scala:331)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:59)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[0],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"json at SparkSQLExample.scala:59","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"mapPartitions\"}","Callsite":"json at SparkSQLExample.scala:59","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"json at SparkSQLExample.scala:59","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"json at SparkSQLExample.scala:59","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameReader.json(DataFrameReader.scala:331)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:59)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788459321,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1547788459627,"Executor ID":"1","Host":"node3","Locality":"NODE_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":1547788459627,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788463652,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"duration total (min, med, max)","Update":"1582","Value":"1581","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":0,"Name":"number of output rows","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"internal.metrics.input.recordsRead","Update":3,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":26,"Name":"internal.metrics.input.bytesRead","Update":73,"Value":73,"Internal":true,"Count Failed Values":true},{"ID":11,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.jvmGCTime","Update":159,"Value":159,"Internal":true,"Count Failed Values":true},{"ID":9,"Name":"internal.metrics.resultSize","Update":1963,"Value":1963,"Internal":true,"Count Failed Values":true},{"ID":8,"Name":"internal.metrics.executorCpuTime","Update":2225128499,"Value":2225128499,"Internal":true,"Count Failed Values":true},{"ID":7,"Name":"internal.metrics.executorRunTime","Update":2291,"Value":2291,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1171850277,"Value":1171850277,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.executorDeserializeTime","Update":1431,"Value":1431,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1431,"Executor Deserialize CPU Time":1171850277,"Executor Run Time":2291,"Executor CPU Time":2225128499,"Result Size":1963,"JVM GC Time":159,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":73,"Records Read":3},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"json at SparkSQLExample.scala:59","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"mapPartitions\"}","Callsite":"json at SparkSQLExample.scala:59","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"json at SparkSQLExample.scala:59","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"json at SparkSQLExample.scala:59","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameReader.json(DataFrameReader.scala:331)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:59)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788459321,"Completion Time":1547788463667,"Accumulables":[{"ID":26,"Name":"internal.metrics.input.bytesRead","Value":73,"Internal":true,"Count Failed Values":true},{"ID":8,"Name":"internal.metrics.executorCpuTime","Value":2225128499,"Internal":true,"Count Failed Values":true},{"ID":11,"Name":"internal.metrics.resultSerializationTime","Value":2,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.executorDeserializeTime","Value":1431,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"duration total (min, med, max)","Value":"1581","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"internal.metrics.executorRunTime","Value":2291,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.jvmGCTime","Value":159,"Internal":true,"Count Failed Values":true},{"ID":27,"Name":"internal.metrics.input.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":9,"Name":"internal.metrics.resultSize","Value":1963,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.executorDeserializeCpuTime","Value":1171850277,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"}]}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1547788463673,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":0,"description":"show at SparkSQLExample.scala:62","details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:62)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#7L as string) AS age#13, cast(name#8 as string) AS name#14]\n +- Relation[age#7L,name#8] json\n\n== Analyzed Logical Plan ==\nage: string, name: string\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#7L as string) AS age#13, cast(name#8 as string) AS name#14]\n +- Relation[age#7L,name#8] json\n\n== Optimized Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#7L as string) AS age#13, name#8]\n +- Relation[age#7L,name#8] json\n\n== Physical Plan ==\nCollectLimit 21\n+- *(1) Project [cast(age#7L as string) AS age#13, name#8]\n +- *(1) FileScan json [age#7L,name#8] Batched: false, DataFilters: [], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n","sparkPlanInfo":{"nodeName":"CollectLimit","simpleString":"CollectLimit 21","children":[{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"Project","simpleString":"Project [cast(age#7L as string) AS age#13, name#8]","children":[{"nodeName":"Scan json ","simpleString":"FileScan json [age#7L,name#8] Batched: false, DataFilters: [], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct","children":[],"metadata":{"Location":"InMemoryFileIndex[hdfs://node6:8020/user/spark/examples/src/main/resources/people.json]","ReadSchema":"struct","Format":"JSON","Batched":"false","PartitionFilters":"[]","PushedFilters":"[]","DataFilters":"[]"},"metrics":[{"name":"number of output rows","accumulatorId":41,"metricType":"sum"},{"name":"number of files","accumulatorId":42,"metricType":"sum"},{"name":"metadata time","accumulatorId":43,"metricType":"sum"},{"name":"scan time total (min, med, max)","accumulatorId":44,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration total (min, med, max)","accumulatorId":40,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":38,"metricType":"sum"},{"name":"shuffle write time total (min, med, max)","accumulatorId":39,"metricType":"nsTiming"},{"name":"records read","accumulatorId":36,"metricType":"sum"},{"name":"local bytes read total (min, med, max)","accumulatorId":34,"metricType":"size"},{"name":"fetch wait time total (min, med, max)","accumulatorId":35,"metricType":"timing"},{"name":"remote bytes read total (min, med, max)","accumulatorId":32,"metricType":"size"},{"name":"local blocks read","accumulatorId":31,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":30,"metricType":"sum"},{"name":"remote bytes read to disk total (min, med, max)","accumulatorId":33,"metricType":"size"},{"name":"shuffle bytes written total (min, med, max)","accumulatorId":37,"metricType":"size"}]},"time":1547788463790} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":0,"accumUpdates":[[42,1],[43,0]]} +{"Event":"SparkListenerJobStart","Job ID":1,"Submission Time":1547788463962,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:62","Number of Tasks":1,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:62","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"FileScanRDD","Scope":"{\"id\":\"6\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:62","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:62","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"6\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:62","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:62)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[1],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"0","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:62","Number of Tasks":1,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:62","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"FileScanRDD","Scope":"{\"id\":\"6\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:62","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:62","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"6\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:62","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:62)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788463966,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"0","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":0,"Attempt":0,"Launch Time":1547788463977,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":0,"Attempt":0,"Launch Time":1547788463977,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788464352,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"duration total (min, med, max)","Update":"172","Value":"171","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":41,"Name":"number of output rows","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":67,"Name":"internal.metrics.input.recordsRead","Update":3,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.input.bytesRead","Update":73,"Value":73,"Internal":true,"Count Failed Values":true},{"ID":49,"Name":"internal.metrics.resultSize","Update":1229,"Value":1229,"Internal":true,"Count Failed Values":true},{"ID":48,"Name":"internal.metrics.executorCpuTime","Update":262646529,"Value":262646529,"Internal":true,"Count Failed Values":true},{"ID":47,"Name":"internal.metrics.executorRunTime","Update":283,"Value":283,"Internal":true,"Count Failed Values":true},{"ID":46,"Name":"internal.metrics.executorDeserializeCpuTime","Update":61019763,"Value":61019763,"Internal":true,"Count Failed Values":true},{"ID":45,"Name":"internal.metrics.executorDeserializeTime","Update":73,"Value":73,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":73,"Executor Deserialize CPU Time":61019763,"Executor Run Time":283,"Executor CPU Time":262646529,"Result Size":1229,"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":73,"Records Read":3},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:62","Number of Tasks":1,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:62","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"FileScanRDD","Scope":"{\"id\":\"6\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:62","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:62","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"6\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:62","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:62)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788463966,"Completion Time":1547788464354,"Accumulables":[{"ID":41,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":47,"Name":"internal.metrics.executorRunTime","Value":283,"Internal":true,"Count Failed Values":true},{"ID":46,"Name":"internal.metrics.executorDeserializeCpuTime","Value":61019763,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"duration total (min, med, max)","Value":"171","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":67,"Name":"internal.metrics.input.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":49,"Name":"internal.metrics.resultSize","Value":1229,"Internal":true,"Count Failed Values":true},{"ID":45,"Name":"internal.metrics.executorDeserializeTime","Value":73,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.input.bytesRead","Value":73,"Internal":true,"Count Failed Values":true},{"ID":48,"Name":"internal.metrics.executorCpuTime","Value":262646529,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":1,"Completion Time":1547788464354,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":0,"time":1547788464363} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":1,"description":"show at SparkSQLExample.scala:82","details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:82)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(name#8 as string) AS name#24]\n +- Project [name#8]\n +- Relation[age#7L,name#8] json\n\n== Analyzed Logical Plan ==\nname: string\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(name#8 as string) AS name#24]\n +- Project [name#8]\n +- Relation[age#7L,name#8] json\n\n== Optimized Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [name#8]\n +- Relation[age#7L,name#8] json\n\n== Physical Plan ==\nCollectLimit 21\n+- *(1) FileScan json [name#8] Batched: false, DataFilters: [], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n","sparkPlanInfo":{"nodeName":"CollectLimit","simpleString":"CollectLimit 21","children":[{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"Scan json ","simpleString":"FileScan json [name#8] Batched: false, DataFilters: [], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct","children":[],"metadata":{"Location":"InMemoryFileIndex[hdfs://node6:8020/user/spark/examples/src/main/resources/people.json]","ReadSchema":"struct","Format":"JSON","Batched":"false","PartitionFilters":"[]","PushedFilters":"[]","DataFilters":"[]"},"metrics":[{"name":"number of output rows","accumulatorId":81,"metricType":"sum"},{"name":"number of files","accumulatorId":82,"metricType":"sum"},{"name":"metadata time","accumulatorId":83,"metricType":"sum"},{"name":"scan time total (min, med, max)","accumulatorId":84,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"duration total (min, med, max)","accumulatorId":80,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":78,"metricType":"sum"},{"name":"shuffle write time total (min, med, max)","accumulatorId":79,"metricType":"nsTiming"},{"name":"records read","accumulatorId":76,"metricType":"sum"},{"name":"local bytes read total (min, med, max)","accumulatorId":74,"metricType":"size"},{"name":"fetch wait time total (min, med, max)","accumulatorId":75,"metricType":"timing"},{"name":"remote bytes read total (min, med, max)","accumulatorId":72,"metricType":"size"},{"name":"local blocks read","accumulatorId":71,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":70,"metricType":"sum"},{"name":"remote bytes read to disk total (min, med, max)","accumulatorId":73,"metricType":"size"},{"name":"shuffle bytes written total (min, med, max)","accumulatorId":77,"metricType":"size"}]},"time":1547788464413} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":1,"accumUpdates":[[82,1],[83,0]]} +{"Event":"SparkListenerJobStart","Job ID":2,"Submission Time":1547788464480,"Stage Infos":[{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:82","Number of Tasks":1,"RDD Info":[{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:82","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"17\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:82","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"15\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:82","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"FileScanRDD","Scope":"{\"id\":\"15\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:82","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:82)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[2],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"1","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:82","Number of Tasks":1,"RDD Info":[{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:82","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"17\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:82","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"15\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:82","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"FileScanRDD","Scope":"{\"id\":\"15\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:82","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:82)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788464483,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"1","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":0,"Attempt":0,"Launch Time":1547788464494,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":0,"Attempt":0,"Launch Time":1547788464494,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788464642,"Failed":false,"Killed":false,"Accumulables":[{"ID":80,"Name":"duration total (min, med, max)","Update":"89","Value":"88","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":81,"Name":"number of output rows","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":107,"Name":"internal.metrics.input.recordsRead","Update":3,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.input.bytesRead","Update":73,"Value":73,"Internal":true,"Count Failed Values":true},{"ID":89,"Name":"internal.metrics.resultSize","Update":1203,"Value":1203,"Internal":true,"Count Failed Values":true},{"ID":88,"Name":"internal.metrics.executorCpuTime","Update":80442300,"Value":80442300,"Internal":true,"Count Failed Values":true},{"ID":87,"Name":"internal.metrics.executorRunTime","Update":96,"Value":96,"Internal":true,"Count Failed Values":true},{"ID":86,"Name":"internal.metrics.executorDeserializeCpuTime","Update":24843191,"Value":24843191,"Internal":true,"Count Failed Values":true},{"ID":85,"Name":"internal.metrics.executorDeserializeTime","Update":35,"Value":35,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":35,"Executor Deserialize CPU Time":24843191,"Executor Run Time":96,"Executor CPU Time":80442300,"Result Size":1203,"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":73,"Records Read":3},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:82","Number of Tasks":1,"RDD Info":[{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:82","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"17\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:82","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"15\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:82","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"FileScanRDD","Scope":"{\"id\":\"15\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:82","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:82)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788464483,"Completion Time":1547788464644,"Accumulables":[{"ID":86,"Name":"internal.metrics.executorDeserializeCpuTime","Value":24843191,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"duration total (min, med, max)","Value":"88","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":107,"Name":"internal.metrics.input.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":89,"Name":"internal.metrics.resultSize","Value":1203,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.input.bytesRead","Value":73,"Internal":true,"Count Failed Values":true},{"ID":85,"Name":"internal.metrics.executorDeserializeTime","Value":35,"Internal":true,"Count Failed Values":true},{"ID":88,"Name":"internal.metrics.executorCpuTime","Value":80442300,"Internal":true,"Count Failed Values":true},{"ID":87,"Name":"internal.metrics.executorRunTime","Value":96,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"}]}} +{"Event":"SparkListenerJobEnd","Job ID":2,"Completion Time":1547788464645,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":1,"time":1547788464646} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":2,"description":"show at SparkSQLExample.scala:92","details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:92)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(name#8 as string) AS name#34, cast((age + 1)#29L as string) AS (age + 1)#35]\n +- Project [name#8, (age#7L + cast(1 as bigint)) AS (age + 1)#29L]\n +- Relation[age#7L,name#8] json\n\n== Analyzed Logical Plan ==\nname: string, (age + 1): string\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(name#8 as string) AS name#34, cast((age + 1)#29L as string) AS (age + 1)#35]\n +- Project [name#8, (age#7L + cast(1 as bigint)) AS (age + 1)#29L]\n +- Relation[age#7L,name#8] json\n\n== Optimized Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [name#8, cast((age#7L + 1) as string) AS (age + 1)#35]\n +- Relation[age#7L,name#8] json\n\n== Physical Plan ==\nCollectLimit 21\n+- *(1) Project [name#8, cast((age#7L + 1) as string) AS (age + 1)#35]\n +- *(1) FileScan json [age#7L,name#8] Batched: false, DataFilters: [], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n","sparkPlanInfo":{"nodeName":"CollectLimit","simpleString":"CollectLimit 21","children":[{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"Project","simpleString":"Project [name#8, cast((age#7L + 1) as string) AS (age + 1)#35]","children":[{"nodeName":"Scan json ","simpleString":"FileScan json [age#7L,name#8] Batched: false, DataFilters: [], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct","children":[],"metadata":{"Location":"InMemoryFileIndex[hdfs://node6:8020/user/spark/examples/src/main/resources/people.json]","ReadSchema":"struct","Format":"JSON","Batched":"false","PartitionFilters":"[]","PushedFilters":"[]","DataFilters":"[]"},"metrics":[{"name":"number of output rows","accumulatorId":121,"metricType":"sum"},{"name":"number of files","accumulatorId":122,"metricType":"sum"},{"name":"metadata time","accumulatorId":123,"metricType":"sum"},{"name":"scan time total (min, med, max)","accumulatorId":124,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration total (min, med, max)","accumulatorId":120,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":118,"metricType":"sum"},{"name":"shuffle write time total (min, med, max)","accumulatorId":119,"metricType":"nsTiming"},{"name":"records read","accumulatorId":116,"metricType":"sum"},{"name":"local bytes read total (min, med, max)","accumulatorId":114,"metricType":"size"},{"name":"fetch wait time total (min, med, max)","accumulatorId":115,"metricType":"timing"},{"name":"remote bytes read total (min, med, max)","accumulatorId":112,"metricType":"size"},{"name":"local blocks read","accumulatorId":111,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":110,"metricType":"sum"},{"name":"remote bytes read to disk total (min, med, max)","accumulatorId":113,"metricType":"size"},{"name":"shuffle bytes written total (min, med, max)","accumulatorId":117,"metricType":"size"}]},"time":1547788464810} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":2,"accumUpdates":[[122,1],[123,3]]} +{"Event":"SparkListenerJobStart","Job ID":3,"Submission Time":1547788464899,"Stage Infos":[{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:92","Number of Tasks":1,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"27\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:92","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"FileScanRDD","Scope":"{\"id\":\"23\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:92","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"23\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:92","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"26\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:92","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:92)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[3],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"2","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:92","Number of Tasks":1,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"27\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:92","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"FileScanRDD","Scope":"{\"id\":\"23\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:92","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"23\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:92","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"26\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:92","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:92)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788464902,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"2","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":0,"Attempt":0,"Launch Time":1547788464910,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":0,"Attempt":0,"Launch Time":1547788464910,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788465075,"Failed":false,"Killed":false,"Accumulables":[{"ID":120,"Name":"duration total (min, med, max)","Update":"86","Value":"85","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":121,"Name":"number of output rows","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":147,"Name":"internal.metrics.input.recordsRead","Update":3,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":146,"Name":"internal.metrics.input.bytesRead","Update":73,"Value":73,"Internal":true,"Count Failed Values":true},{"ID":129,"Name":"internal.metrics.resultSize","Update":1234,"Value":1234,"Internal":true,"Count Failed Values":true},{"ID":128,"Name":"internal.metrics.executorCpuTime","Update":109502826,"Value":109502826,"Internal":true,"Count Failed Values":true},{"ID":127,"Name":"internal.metrics.executorRunTime","Update":127,"Value":127,"Internal":true,"Count Failed Values":true},{"ID":126,"Name":"internal.metrics.executorDeserializeCpuTime","Update":17361720,"Value":17361720,"Internal":true,"Count Failed Values":true},{"ID":125,"Name":"internal.metrics.executorDeserializeTime","Update":24,"Value":24,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":24,"Executor Deserialize CPU Time":17361720,"Executor Run Time":127,"Executor CPU Time":109502826,"Result Size":1234,"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":73,"Records Read":3},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:92","Number of Tasks":1,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"27\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:92","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"FileScanRDD","Scope":"{\"id\":\"23\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:92","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"23\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:92","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"26\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:92","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:92)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788464902,"Completion Time":1547788465077,"Accumulables":[{"ID":146,"Name":"internal.metrics.input.bytesRead","Value":73,"Internal":true,"Count Failed Values":true},{"ID":128,"Name":"internal.metrics.executorCpuTime","Value":109502826,"Internal":true,"Count Failed Values":true},{"ID":125,"Name":"internal.metrics.executorDeserializeTime","Value":24,"Internal":true,"Count Failed Values":true},{"ID":127,"Name":"internal.metrics.executorRunTime","Value":127,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":120,"Name":"duration total (min, med, max)","Value":"85","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":147,"Name":"internal.metrics.input.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":129,"Name":"internal.metrics.resultSize","Value":1234,"Internal":true,"Count Failed Values":true},{"ID":126,"Name":"internal.metrics.executorDeserializeCpuTime","Value":17361720,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":3,"Completion Time":1547788465078,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":2,"time":1547788465079} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":3,"description":"show at SparkSQLExample.scala:102","details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:102)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#7L as string) AS age#45, cast(name#8 as string) AS name#46]\n +- Filter (age#7L > cast(21 as bigint))\n +- Relation[age#7L,name#8] json\n\n== Analyzed Logical Plan ==\nage: string, name: string\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#7L as string) AS age#45, cast(name#8 as string) AS name#46]\n +- Filter (age#7L > cast(21 as bigint))\n +- Relation[age#7L,name#8] json\n\n== Optimized Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#7L as string) AS age#45, name#8]\n +- Filter (isnotnull(age#7L) && (age#7L > 21))\n +- Relation[age#7L,name#8] json\n\n== Physical Plan ==\nCollectLimit 21\n+- *(1) Project [cast(age#7L as string) AS age#45, name#8]\n +- *(1) Filter (isnotnull(age#7L) && (age#7L > 21))\n +- *(1) FileScan json [age#7L,name#8] Batched: false, DataFilters: [isnotnull(age#7L), (age#7L > 21)], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [IsNotNull(age), GreaterThan(age,21)], ReadSchema: struct\n","sparkPlanInfo":{"nodeName":"CollectLimit","simpleString":"CollectLimit 21","children":[{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"Project","simpleString":"Project [cast(age#7L as string) AS age#45, name#8]","children":[{"nodeName":"Filter","simpleString":"Filter (isnotnull(age#7L) && (age#7L > 21))","children":[{"nodeName":"Scan json ","simpleString":"FileScan json [age#7L,name#8] Batched: false, DataFilters: [isnotnull(age#7L), (age#7L > 21)], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [IsNotNull(age), GreaterThan(age,21)], ReadSchema: struct","children":[],"metadata":{"Location":"InMemoryFileIndex[hdfs://node6:8020/user/spark/examples/src/main/resources/people.json]","ReadSchema":"struct","Format":"JSON","Batched":"false","PartitionFilters":"[]","PushedFilters":"[IsNotNull(age), GreaterThan(age,21)]","DataFilters":"[isnotnull(age#7L), (age#7L > 21)]"},"metrics":[{"name":"number of output rows","accumulatorId":162,"metricType":"sum"},{"name":"number of files","accumulatorId":163,"metricType":"sum"},{"name":"metadata time","accumulatorId":164,"metricType":"sum"},{"name":"scan time total (min, med, max)","accumulatorId":165,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":161,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration total (min, med, max)","accumulatorId":160,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":158,"metricType":"sum"},{"name":"shuffle write time total (min, med, max)","accumulatorId":159,"metricType":"nsTiming"},{"name":"records read","accumulatorId":156,"metricType":"sum"},{"name":"local bytes read total (min, med, max)","accumulatorId":154,"metricType":"size"},{"name":"fetch wait time total (min, med, max)","accumulatorId":155,"metricType":"timing"},{"name":"remote bytes read total (min, med, max)","accumulatorId":152,"metricType":"size"},{"name":"local blocks read","accumulatorId":151,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":150,"metricType":"sum"},{"name":"remote bytes read to disk total (min, med, max)","accumulatorId":153,"metricType":"size"},{"name":"shuffle bytes written total (min, med, max)","accumulatorId":157,"metricType":"size"}]},"time":1547788465156} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":3,"accumUpdates":[[163,1],[164,0]]} +{"Event":"SparkListenerJobStart","Job ID":4,"Submission Time":1547788465226,"Stage Infos":[{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:102","Number of Tasks":1,"RDD Info":[{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"37\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:102","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"32\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:102","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"FileScanRDD","Scope":"{\"id\":\"32\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:102","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"36\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:102","Parent IDs":[16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:102)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[4],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"3","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:102","Number of Tasks":1,"RDD Info":[{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"37\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:102","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"32\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:102","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"FileScanRDD","Scope":"{\"id\":\"32\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:102","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"36\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:102","Parent IDs":[16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:102)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788465227,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"3","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":0,"Attempt":0,"Launch Time":1547788465233,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":0,"Attempt":0,"Launch Time":1547788465233,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788465370,"Failed":false,"Killed":false,"Accumulables":[{"ID":160,"Name":"duration total (min, med, max)","Update":"56","Value":"55","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":161,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":162,"Name":"number of output rows","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":188,"Name":"internal.metrics.input.recordsRead","Update":3,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":187,"Name":"internal.metrics.input.bytesRead","Update":73,"Value":73,"Internal":true,"Count Failed Values":true},{"ID":170,"Name":"internal.metrics.resultSize","Update":1247,"Value":1247,"Internal":true,"Count Failed Values":true},{"ID":169,"Name":"internal.metrics.executorCpuTime","Update":83184993,"Value":83184993,"Internal":true,"Count Failed Values":true},{"ID":168,"Name":"internal.metrics.executorRunTime","Update":95,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":167,"Name":"internal.metrics.executorDeserializeCpuTime","Update":18971260,"Value":18971260,"Internal":true,"Count Failed Values":true},{"ID":166,"Name":"internal.metrics.executorDeserializeTime","Update":28,"Value":28,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":28,"Executor Deserialize CPU Time":18971260,"Executor Run Time":95,"Executor CPU Time":83184993,"Result Size":1247,"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":73,"Records Read":3},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:102","Number of Tasks":1,"RDD Info":[{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"37\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:102","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"32\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:102","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"FileScanRDD","Scope":"{\"id\":\"32\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:102","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"36\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:102","Parent IDs":[16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:102)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788465227,"Completion Time":1547788465372,"Accumulables":[{"ID":167,"Name":"internal.metrics.executorDeserializeCpuTime","Value":18971260,"Internal":true,"Count Failed Values":true},{"ID":166,"Name":"internal.metrics.executorDeserializeTime","Value":28,"Internal":true,"Count Failed Values":true},{"ID":160,"Name":"duration total (min, med, max)","Value":"55","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":187,"Name":"internal.metrics.input.bytesRead","Value":73,"Internal":true,"Count Failed Values":true},{"ID":169,"Name":"internal.metrics.executorCpuTime","Value":83184993,"Internal":true,"Count Failed Values":true},{"ID":168,"Name":"internal.metrics.executorRunTime","Value":95,"Internal":true,"Count Failed Values":true},{"ID":162,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":161,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":188,"Name":"internal.metrics.input.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":170,"Name":"internal.metrics.resultSize","Value":1247,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":4,"Completion Time":1547788465372,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":3,"time":1547788465374} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":4,"description":"show at SparkSQLExample.scala:110","details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#7L as string) AS age#62, cast(count#57L as string) AS count#63]\n +- Aggregate [age#7L], [age#7L, count(1) AS count#57L]\n +- Relation[age#7L,name#8] json\n\n== Analyzed Logical Plan ==\nage: string, count: string\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#7L as string) AS age#62, cast(count#57L as string) AS count#63]\n +- Aggregate [age#7L], [age#7L, count(1) AS count#57L]\n +- Relation[age#7L,name#8] json\n\n== Optimized Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Aggregate [age#7L], [cast(age#7L as string) AS age#62, cast(count(1) as string) AS count#63]\n +- Project [age#7L]\n +- Relation[age#7L,name#8] json\n\n== Physical Plan ==\nCollectLimit 21\n+- *(2) HashAggregate(keys=[age#7L], functions=[count(1)], output=[age#62, count#63])\n +- Exchange hashpartitioning(age#7L, 200)\n +- *(1) HashAggregate(keys=[age#7L], functions=[partial_count(1)], output=[age#7L, count#67L])\n +- *(1) FileScan json [age#7L] Batched: false, DataFilters: [], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n","sparkPlanInfo":{"nodeName":"CollectLimit","simpleString":"CollectLimit 21","children":[{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[age#7L], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(age#7L, 200)","children":[{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[age#7L], functions=[partial_count(1)])","children":[{"nodeName":"Scan json ","simpleString":"FileScan json [age#7L] Batched: false, DataFilters: [], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct","children":[],"metadata":{"Location":"InMemoryFileIndex[hdfs://node6:8020/user/spark/examples/src/main/resources/people.json]","ReadSchema":"struct","Format":"JSON","Batched":"false","PartitionFilters":"[]","PushedFilters":"[]","DataFilters":"[]"},"metrics":[{"name":"number of output rows","accumulatorId":235,"metricType":"sum"},{"name":"number of files","accumulatorId":236,"metricType":"sum"},{"name":"metadata time","accumulatorId":237,"metricType":"sum"},{"name":"scan time total (min, med, max)","accumulatorId":238,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"spill size total (min, med, max)","accumulatorId":232,"metricType":"size"},{"name":"aggregate time total (min, med, max)","accumulatorId":233,"metricType":"timing"},{"name":"peak memory total (min, med, max)","accumulatorId":231,"metricType":"size"},{"name":"number of output rows","accumulatorId":230,"metricType":"sum"},{"name":"avg hash probe (min, med, max)","accumulatorId":234,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration total (min, med, max)","accumulatorId":229,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":211,"metricType":"sum"},{"name":"shuffle write time total (min, med, max)","accumulatorId":212,"metricType":"nsTiming"},{"name":"records read","accumulatorId":209,"metricType":"sum"},{"name":"local bytes read total (min, med, max)","accumulatorId":207,"metricType":"size"},{"name":"fetch wait time total (min, med, max)","accumulatorId":208,"metricType":"timing"},{"name":"remote bytes read total (min, med, max)","accumulatorId":205,"metricType":"size"},{"name":"local blocks read","accumulatorId":204,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":203,"metricType":"sum"},{"name":"data size total (min, med, max)","accumulatorId":202,"metricType":"size"},{"name":"remote bytes read to disk total (min, med, max)","accumulatorId":206,"metricType":"size"},{"name":"shuffle bytes written total (min, med, max)","accumulatorId":210,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size total (min, med, max)","accumulatorId":226,"metricType":"size"},{"name":"aggregate time total (min, med, max)","accumulatorId":227,"metricType":"timing"},{"name":"peak memory total (min, med, max)","accumulatorId":225,"metricType":"size"},{"name":"number of output rows","accumulatorId":224,"metricType":"sum"},{"name":"avg hash probe (min, med, max)","accumulatorId":228,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration total (min, med, max)","accumulatorId":223,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":221,"metricType":"sum"},{"name":"shuffle write time total (min, med, max)","accumulatorId":222,"metricType":"nsTiming"},{"name":"records read","accumulatorId":219,"metricType":"sum"},{"name":"local bytes read total (min, med, max)","accumulatorId":217,"metricType":"size"},{"name":"fetch wait time total (min, med, max)","accumulatorId":218,"metricType":"timing"},{"name":"remote bytes read total (min, med, max)","accumulatorId":215,"metricType":"size"},{"name":"local blocks read","accumulatorId":214,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":213,"metricType":"sum"},{"name":"remote bytes read to disk total (min, med, max)","accumulatorId":216,"metricType":"size"},{"name":"shuffle bytes written total (min, med, max)","accumulatorId":220,"metricType":"size"}]},"time":1547788465472} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":4,"accumUpdates":[[236,1],[237,0]]} +{"Event":"SparkListenerJobStart","Job ID":5,"Submission Time":1547788465753,"Stage Infos":[{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":1,"RDD Info":[{"RDD ID":21,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":19,"Name":"FileScanRDD","Scope":"{\"id\":\"46\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"46\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]},{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":1,"RDD Info":[{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"50\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[5],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[5,6],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"4","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":1,"RDD Info":[{"RDD ID":21,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":19,"Name":"FileScanRDD","Scope":"{\"id\":\"46\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"46\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788465757,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"4","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":0,"Attempt":0,"Launch Time":1547788465770,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":0,"Attempt":0,"Launch Time":1547788465770,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788466838,"Failed":false,"Killed":false,"Accumulables":[{"ID":212,"Name":"shuffle write time total (min, med, max)","Update":"59882301","Value":"59882300","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":211,"Name":"shuffle records written","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":210,"Name":"shuffle bytes written total (min, med, max)","Update":"185","Value":"184","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":202,"Name":"data size total (min, med, max)","Update":"71","Value":"70","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":229,"Name":"duration total (min, med, max)","Update":"633","Value":"632","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":234,"Name":"avg hash probe (min, med, max)","Update":"10","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":230,"Name":"number of output rows","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":231,"Name":"peak memory total (min, med, max)","Update":"17039359","Value":"17039358","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":233,"Name":"aggregate time total (min, med, max)","Update":"170","Value":"169","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":235,"Name":"number of output rows","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":261,"Name":"internal.metrics.input.recordsRead","Update":3,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":260,"Name":"internal.metrics.input.bytesRead","Update":73,"Value":73,"Internal":true,"Count Failed Values":true},{"ID":259,"Name":"internal.metrics.shuffle.write.writeTime","Update":59882302,"Value":59882302,"Internal":true,"Count Failed Values":true},{"ID":258,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":3,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":257,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":186,"Value":186,"Internal":true,"Count Failed Values":true},{"ID":248,"Name":"internal.metrics.peakExecutionMemory","Update":17039360,"Value":17039360,"Internal":true,"Count Failed Values":true},{"ID":245,"Name":"internal.metrics.resultSerializationTime","Update":3,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":243,"Name":"internal.metrics.resultSize","Update":2687,"Value":2687,"Internal":true,"Count Failed Values":true},{"ID":242,"Name":"internal.metrics.executorCpuTime","Update":710856733,"Value":710856733,"Internal":true,"Count Failed Values":true},{"ID":241,"Name":"internal.metrics.executorRunTime","Update":871,"Value":871,"Internal":true,"Count Failed Values":true},{"ID":240,"Name":"internal.metrics.executorDeserializeCpuTime","Update":162794413,"Value":162794413,"Internal":true,"Count Failed Values":true},{"ID":239,"Name":"internal.metrics.executorDeserializeTime","Update":177,"Value":177,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":177,"Executor Deserialize CPU Time":162794413,"Executor Run Time":871,"Executor CPU Time":710856733,"Result Size":2687,"JVM GC Time":0,"Result Serialization Time":3,"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":186,"Shuffle Write Time":59882302,"Shuffle Records Written":3},"Input Metrics":{"Bytes Read":73,"Records Read":3},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":1,"RDD Info":[{"RDD ID":21,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":19,"Name":"FileScanRDD","Scope":"{\"id\":\"46\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"46\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788465757,"Completion Time":1547788466840,"Accumulables":[{"ID":245,"Name":"internal.metrics.resultSerializationTime","Value":3,"Internal":true,"Count Failed Values":true},{"ID":235,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":211,"Name":"shuffle records written","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":202,"Name":"data size total (min, med, max)","Value":"70","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":229,"Name":"duration total (min, med, max)","Value":"632","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":259,"Name":"internal.metrics.shuffle.write.writeTime","Value":59882302,"Internal":true,"Count Failed Values":true},{"ID":241,"Name":"internal.metrics.executorRunTime","Value":871,"Internal":true,"Count Failed Values":true},{"ID":240,"Name":"internal.metrics.executorDeserializeCpuTime","Value":162794413,"Internal":true,"Count Failed Values":true},{"ID":258,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":3,"Internal":true,"Count Failed Values":true},{"ID":234,"Name":"avg hash probe (min, med, max)","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":243,"Name":"internal.metrics.resultSize","Value":2687,"Internal":true,"Count Failed Values":true},{"ID":261,"Name":"internal.metrics.input.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":210,"Name":"shuffle bytes written total (min, med, max)","Value":"184","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":231,"Name":"peak memory total (min, med, max)","Value":"17039358","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":230,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":257,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":186,"Internal":true,"Count Failed Values":true},{"ID":212,"Name":"shuffle write time total (min, med, max)","Value":"59882300","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":239,"Name":"internal.metrics.executorDeserializeTime","Value":177,"Internal":true,"Count Failed Values":true},{"ID":248,"Name":"internal.metrics.peakExecutionMemory","Value":17039360,"Internal":true,"Count Failed Values":true},{"ID":260,"Name":"internal.metrics.input.bytesRead","Value":73,"Internal":true,"Count Failed Values":true},{"ID":233,"Name":"aggregate time total (min, med, max)","Value":"169","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":242,"Name":"internal.metrics.executorCpuTime","Value":710856733,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":1,"RDD Info":[{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"50\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[5],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788466853,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"4","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":0,"Attempt":0,"Launch Time":1547788466859,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":0,"Attempt":0,"Launch Time":1547788466859,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788467141,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"16","Value":"15","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"262142","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":227,"Name":"aggregate time total (min, med, max)","Update":"1","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":281,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":280,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":279,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":278,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":277,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":276,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":275,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":273,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":262144,"Internal":true,"Count Failed Values":true},{"ID":268,"Name":"internal.metrics.resultSize","Update":3617,"Value":3617,"Internal":true,"Count Failed Values":true},{"ID":267,"Name":"internal.metrics.executorCpuTime","Update":172136588,"Value":172136588,"Internal":true,"Count Failed Values":true},{"ID":266,"Name":"internal.metrics.executorRunTime","Update":192,"Value":192,"Internal":true,"Count Failed Values":true},{"ID":265,"Name":"internal.metrics.executorDeserializeCpuTime","Update":65342261,"Value":65342261,"Internal":true,"Count Failed Values":true},{"ID":264,"Name":"internal.metrics.executorDeserializeTime","Update":76,"Value":76,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":76,"Executor Deserialize CPU Time":65342261,"Executor Run Time":192,"Executor CPU Time":172136588,"Result Size":3617,"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":"SparkListenerStageCompleted","Stage Info":{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":1,"RDD Info":[{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"50\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[5],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788466853,"Completion Time":1547788467143,"Accumulables":[{"ID":227,"Name":"aggregate time total (min, med, max)","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":280,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":265,"Name":"internal.metrics.executorDeserializeCpuTime","Value":65342261,"Internal":true,"Count Failed Values":true},{"ID":223,"Name":"duration total (min, med, max)","Value":"15","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":277,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":268,"Name":"internal.metrics.resultSize","Value":3617,"Internal":true,"Count Failed Values":true},{"ID":267,"Name":"internal.metrics.executorCpuTime","Value":172136588,"Internal":true,"Count Failed Values":true},{"ID":276,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":225,"Name":"peak memory total (min, med, max)","Value":"262142","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":279,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":264,"Name":"internal.metrics.executorDeserializeTime","Value":76,"Internal":true,"Count Failed Values":true},{"ID":273,"Name":"internal.metrics.peakExecutionMemory","Value":262144,"Internal":true,"Count Failed Values":true},{"ID":281,"Name":"internal.metrics.shuffle.read.recordsRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":266,"Name":"internal.metrics.executorRunTime","Value":192,"Internal":true,"Count Failed Values":true},{"ID":275,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":278,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":5,"Completion Time":1547788467143,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":6,"Submission Time":1547788467155,"Stage Infos":[{"Stage ID":7,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":1,"RDD Info":[{"RDD ID":21,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":19,"Name":"FileScanRDD","Scope":"{\"id\":\"46\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"46\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]},{"Stage ID":8,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":4,"RDD Info":[{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"50\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[7],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[7,8],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"4","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":8,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":4,"RDD Info":[{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"50\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[7],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788467158,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"4","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerTaskStart","Stage ID":8,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":3,"Attempt":0,"Launch Time":1547788467166,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":8,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1547788467167,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":8,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1547788467255,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":8,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":3,"Attempt":0,"Launch Time":1547788467166,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788467256,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"29","Value":"44","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":228,"Name":"avg hash probe (min, med, max)","Update":"10","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":224,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"17039359","Value":"17301501","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":227,"Name":"aggregate time total (min, med, max)","Update":"19","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":204,"Name":"local blocks read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":207,"Name":"local bytes read total (min, med, max)","Update":"62","Value":"60","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":209,"Name":"records read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":306,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":305,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":304,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":63,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":303,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":302,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":301,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":300,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":298,"Name":"internal.metrics.peakExecutionMemory","Update":17039360,"Value":17039360,"Internal":true,"Count Failed Values":true},{"ID":293,"Name":"internal.metrics.resultSize","Update":3656,"Value":3656,"Internal":true,"Count Failed Values":true},{"ID":292,"Name":"internal.metrics.executorCpuTime","Update":50963610,"Value":50963610,"Internal":true,"Count Failed Values":true},{"ID":291,"Name":"internal.metrics.executorRunTime","Update":51,"Value":51,"Internal":true,"Count Failed Values":true},{"ID":290,"Name":"internal.metrics.executorDeserializeCpuTime","Update":18570925,"Value":18570925,"Internal":true,"Count Failed Values":true},{"ID":289,"Name":"internal.metrics.executorDeserializeTime","Update":25,"Value":25,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":25,"Executor Deserialize CPU Time":18570925,"Executor Run Time":51,"Executor CPU Time":50963610,"Result Size":3656,"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":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":63,"Total Records Read":1},"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":8,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1547788467301,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":8,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1547788467255,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788467302,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"14","Value":"58","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"17563644","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":306,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":305,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":304,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":303,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":302,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":301,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":300,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":298,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":17301504,"Internal":true,"Count Failed Values":true},{"ID":293,"Name":"internal.metrics.resultSize","Update":3617,"Value":7273,"Internal":true,"Count Failed Values":true},{"ID":292,"Name":"internal.metrics.executorCpuTime","Update":20052870,"Value":71016480,"Internal":true,"Count Failed Values":true},{"ID":291,"Name":"internal.metrics.executorRunTime","Update":20,"Value":71,"Internal":true,"Count Failed Values":true},{"ID":290,"Name":"internal.metrics.executorDeserializeCpuTime","Update":13903504,"Value":32474429,"Internal":true,"Count Failed Values":true},{"ID":289,"Name":"internal.metrics.executorDeserializeTime","Update":14,"Value":39,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":14,"Executor Deserialize CPU Time":13903504,"Executor Run Time":20,"Executor CPU Time":20052870,"Result Size":3617,"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":8,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1547788467301,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788467346,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"17","Value":"75","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"17825787","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":306,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":305,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":304,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":303,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":302,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":301,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":300,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":298,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":17563648,"Internal":true,"Count Failed Values":true},{"ID":293,"Name":"internal.metrics.resultSize","Update":3617,"Value":10890,"Internal":true,"Count Failed Values":true},{"ID":292,"Name":"internal.metrics.executorCpuTime","Update":17420764,"Value":88437244,"Internal":true,"Count Failed Values":true},{"ID":291,"Name":"internal.metrics.executorRunTime","Update":22,"Value":93,"Internal":true,"Count Failed Values":true},{"ID":290,"Name":"internal.metrics.executorDeserializeCpuTime","Update":11738163,"Value":44212592,"Internal":true,"Count Failed Values":true},{"ID":289,"Name":"internal.metrics.executorDeserializeTime","Update":13,"Value":52,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":13,"Executor Deserialize CPU Time":11738163,"Executor Run Time":22,"Executor CPU Time":17420764,"Result Size":3617,"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":8,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1547788467167,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788470544,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"209","Value":"284","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"18087930","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":227,"Name":"aggregate time total (min, med, max)","Update":"8","Value":"25","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":306,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":305,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":304,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":303,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":302,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":301,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":300,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":298,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":17825792,"Internal":true,"Count Failed Values":true},{"ID":295,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":294,"Name":"internal.metrics.jvmGCTime","Update":162,"Value":162,"Internal":true,"Count Failed Values":true},{"ID":293,"Name":"internal.metrics.resultSize","Update":3703,"Value":14593,"Internal":true,"Count Failed Values":true},{"ID":292,"Name":"internal.metrics.executorCpuTime","Update":1088992840,"Value":1177430084,"Internal":true,"Count Failed Values":true},{"ID":291,"Name":"internal.metrics.executorRunTime","Update":1330,"Value":1423,"Internal":true,"Count Failed Values":true},{"ID":290,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1446326008,"Value":1490538600,"Internal":true,"Count Failed Values":true},{"ID":289,"Name":"internal.metrics.executorDeserializeTime","Update":1996,"Value":2048,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1996,"Executor Deserialize CPU Time":1446326008,"Executor Run Time":1330,"Executor CPU Time":1088992840,"Result Size":3703,"JVM GC Time":162,"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":"SparkListenerStageCompleted","Stage Info":{"Stage ID":8,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":4,"RDD Info":[{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"50\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[7],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788467158,"Completion Time":1547788470547,"Accumulables":[{"ID":227,"Name":"aggregate time total (min, med, max)","Value":"25","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":209,"Name":"records read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":304,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":63,"Internal":true,"Count Failed Values":true},{"ID":295,"Name":"internal.metrics.resultSerializationTime","Value":1,"Internal":true,"Count Failed Values":true},{"ID":298,"Name":"internal.metrics.peakExecutionMemory","Value":17825792,"Internal":true,"Count Failed Values":true},{"ID":289,"Name":"internal.metrics.executorDeserializeTime","Value":2048,"Internal":true,"Count Failed Values":true},{"ID":301,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":292,"Name":"internal.metrics.executorCpuTime","Value":1177430084,"Internal":true,"Count Failed Values":true},{"ID":223,"Name":"duration total (min, med, max)","Value":"284","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":300,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":294,"Name":"internal.metrics.jvmGCTime","Value":162,"Internal":true,"Count Failed Values":true},{"ID":303,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":225,"Name":"peak memory total (min, med, max)","Value":"18087930","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":306,"Name":"internal.metrics.shuffle.read.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":207,"Name":"local bytes read total (min, med, max)","Value":"60","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":228,"Name":"avg hash probe (min, med, max)","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":291,"Name":"internal.metrics.executorRunTime","Value":1423,"Internal":true,"Count Failed Values":true},{"ID":204,"Name":"local blocks read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":290,"Name":"internal.metrics.executorDeserializeCpuTime","Value":1490538600,"Internal":true,"Count Failed Values":true},{"ID":293,"Name":"internal.metrics.resultSize","Value":14593,"Internal":true,"Count Failed Values":true},{"ID":302,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":224,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":305,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":6,"Completion Time":1547788470548,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":7,"Submission Time":1547788470558,"Stage Infos":[{"Stage ID":9,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":1,"RDD Info":[{"RDD ID":21,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":19,"Name":"FileScanRDD","Scope":"{\"id\":\"46\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"46\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]},{"Stage ID":10,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":20,"RDD Info":[{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"50\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[9],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[9,10],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"4","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":10,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":20,"RDD Info":[{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"50\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[9],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788470562,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"4","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerTaskStart","Stage ID":10,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":0,"Attempt":0,"Launch Time":1547788470568,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":10,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":1,"Attempt":0,"Launch Time":1547788470569,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":10,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":2,"Attempt":0,"Launch Time":1547788470626,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":0,"Attempt":0,"Launch Time":1547788470568,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788470627,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"13","Value":"297","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"18350073","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":262144,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3617,"Value":3617,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":18437250,"Value":18437250,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":18,"Value":18,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":20743656,"Value":20743656,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":29,"Value":29,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":29,"Executor Deserialize CPU Time":20743656,"Executor Run Time":18,"Executor CPU Time":18437250,"Result Size":3617,"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":10,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":3,"Attempt":0,"Launch Time":1547788470668,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":2,"Attempt":0,"Launch Time":1547788470626,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788470668,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"14","Value":"311","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"18612216","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3617,"Value":7234,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":19340397,"Value":37777647,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":18,"Value":36,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":12502299,"Value":33245955,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":13,"Value":42,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":13,"Executor Deserialize CPU Time":12502299,"Executor Run Time":18,"Executor CPU Time":19340397,"Result Size":3617,"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":10,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":4,"Attempt":0,"Launch Time":1547788470699,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":3,"Attempt":0,"Launch Time":1547788470668,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788470699,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"9","Value":"320","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"18874359","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":786432,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3617,"Value":10851,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":13392141,"Value":51169788,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":14,"Value":50,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8423156,"Value":41669111,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":8,"Value":50,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":8,"Executor Deserialize CPU Time":8423156,"Executor Run Time":14,"Executor CPU Time":13392141,"Result Size":3617,"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":10,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":5,"Attempt":0,"Launch Time":1547788470732,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":4,"Attempt":0,"Launch Time":1547788470699,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788470733,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"9","Value":"329","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"19136502","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":1048576,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3617,"Value":14468,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":13950272,"Value":65120060,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":14,"Value":64,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":10176458,"Value":51845569,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":10,"Value":60,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":10,"Executor Deserialize CPU Time":10176458,"Executor Run Time":14,"Executor CPU Time":13950272,"Result Size":3617,"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":10,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":6,"Attempt":0,"Launch Time":1547788470767,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":5,"Attempt":0,"Launch Time":1547788470732,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788470767,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"10","Value":"339","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"19398645","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":1310720,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3617,"Value":18085,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":15018675,"Value":80138735,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":15,"Value":79,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":10862497,"Value":62708066,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":11,"Value":71,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":11,"Executor Deserialize CPU Time":10862497,"Executor Run Time":15,"Executor CPU Time":15018675,"Result Size":3617,"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":10,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":7,"Attempt":0,"Launch Time":1547788470799,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":6,"Attempt":0,"Launch Time":1547788470767,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788470799,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"7","Value":"346","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"19660788","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":1572864,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3617,"Value":21702,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":12762090,"Value":92900825,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":13,"Value":92,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":10447486,"Value":73155552,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":10,"Value":81,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":10,"Executor Deserialize CPU Time":10447486,"Executor Run Time":13,"Executor CPU Time":12762090,"Result Size":3617,"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":10,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":8,"Attempt":0,"Launch Time":1547788470824,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":7,"Attempt":0,"Launch Time":1547788470799,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788470825,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"352","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"19922931","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":1835008,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3617,"Value":25319,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":9882914,"Value":102783739,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":10,"Value":102,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7633805,"Value":80789357,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":88,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":7633805,"Executor Run Time":10,"Executor CPU Time":9882914,"Result Size":3617,"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":10,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":9,"Attempt":0,"Launch Time":1547788470846,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":8,"Attempt":0,"Launch Time":1547788470824,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788470847,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"357","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"20185074","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":2097152,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3617,"Value":28936,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":9266539,"Value":112050278,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":9,"Value":111,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6207804,"Value":86997161,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":94,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":6207804,"Executor Run Time":9,"Executor CPU Time":9266539,"Result Size":3617,"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":10,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":10,"Attempt":0,"Launch Time":1547788470866,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":1,"Attempt":0,"Launch Time":1547788470569,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788470867,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"34","Value":"391","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"20447217","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":227,"Name":"aggregate time total (min, med, max)","Update":"1","Value":"18","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":2359296,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3617,"Value":32553,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":34825296,"Value":146875574,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":45,"Value":156,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":35461794,"Value":122458955,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":236,"Value":330,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":236,"Executor Deserialize CPU Time":35461794,"Executor Run Time":45,"Executor CPU Time":34825296,"Result Size":3617,"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":10,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":11,"Attempt":0,"Launch Time":1547788470869,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":9,"Attempt":0,"Launch Time":1547788470846,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788470870,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"7","Value":"398","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"20709360","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":2621440,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3617,"Value":36170,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":9989200,"Value":156864774,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":10,"Value":166,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5843063,"Value":128302018,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":336,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5843063,"Executor Run Time":10,"Executor CPU Time":9989200,"Result Size":3617,"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":10,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":12,"Attempt":0,"Launch Time":1547788470891,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":11,"Attempt":0,"Launch Time":1547788470869,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788470892,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"403","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"20971503","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":2883584,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3617,"Value":39787,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":8835589,"Value":165700363,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":9,"Value":175,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5785895,"Value":134087913,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":341,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5785895,"Executor Run Time":9,"Executor CPU Time":8835589,"Result Size":3617,"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":10,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":13,"Attempt":0,"Launch Time":1547788470917,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":12,"Attempt":0,"Launch Time":1547788470891,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788470918,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"7","Value":"410","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"21233646","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":3145728,"Internal":true,"Count Failed Values":true},{"ID":320,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3660,"Value":43447,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":9468324,"Value":175168687,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":10,"Value":185,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7086416,"Value":141174329,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":348,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":7086416,"Executor Run Time":10,"Executor CPU Time":9468324,"Result Size":3660,"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":10,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":14,"Attempt":0,"Launch Time":1547788470930,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":10,"Attempt":0,"Launch Time":1547788470866,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788470930,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"27","Value":"437","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"21495789","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":3407872,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3617,"Value":47064,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":23223894,"Value":198392581,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":33,"Value":218,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":12667423,"Value":153841752,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":20,"Value":368,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":20,"Executor Deserialize CPU Time":12667423,"Executor Run Time":33,"Executor CPU Time":23223894,"Result Size":3617,"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":10,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":15,"Attempt":0,"Launch Time":1547788470939,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":13,"Attempt":0,"Launch Time":1547788470917,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788470940,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"442","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"21757932","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":3670016,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3617,"Value":50681,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":9181493,"Value":207574074,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":9,"Value":227,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6095760,"Value":159937512,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":374,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":6095760,"Executor Run Time":9,"Executor CPU Time":9181493,"Result Size":3617,"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":10,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":16,"Attempt":0,"Launch Time":1547788470961,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":15,"Attempt":0,"Launch Time":1547788470939,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788470962,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"448","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"22020075","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":3932160,"Internal":true,"Count Failed Values":true},{"ID":320,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3660,"Value":54341,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":9000681,"Value":216574755,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":9,"Value":236,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5739878,"Value":165677390,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":380,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5739878,"Executor Run Time":9,"Executor CPU Time":9000681,"Result Size":3660,"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":10,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":17,"Attempt":0,"Launch Time":1547788470983,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":16,"Attempt":0,"Launch Time":1547788470961,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788470984,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"454","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"22282218","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":4194304,"Internal":true,"Count Failed Values":true},{"ID":320,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3660,"Value":58001,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":9278513,"Value":225853268,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":9,"Value":245,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5693904,"Value":171371294,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":386,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5693904,"Executor Run Time":9,"Executor CPU Time":9278513,"Result Size":3660,"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":10,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":18,"Attempt":0,"Launch Time":1547788471004,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":17,"Attempt":0,"Launch Time":1547788470983,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471005,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"459","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"22544361","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":4456448,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3617,"Value":61618,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":8992812,"Value":234846080,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":9,"Value":254,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5709977,"Value":177081271,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":392,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5709977,"Executor Run Time":9,"Executor CPU Time":8992812,"Result Size":3617,"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":10,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":19,"Attempt":0,"Launch Time":1547788471006,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":14,"Attempt":0,"Launch Time":1547788470930,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471006,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"42","Value":"501","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"22806504","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":227,"Name":"aggregate time total (min, med, max)","Update":"15","Value":"25","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":4718592,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3617,"Value":65235,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":20343990,"Value":255190070,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":51,"Value":305,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":9788084,"Value":186869355,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":11,"Value":403,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":11,"Executor Deserialize CPU Time":9788084,"Executor Run Time":51,"Executor CPU Time":20343990,"Result Size":3617,"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":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":18,"Attempt":0,"Launch Time":1547788471004,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471026,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"506","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"23068647","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":4980736,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3617,"Value":68852,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":8887889,"Value":264077959,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":9,"Value":314,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5985079,"Value":192854434,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":409,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5985079,"Executor Run Time":9,"Executor CPU Time":8887889,"Result Size":3617,"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":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":19,"Attempt":0,"Launch Time":1547788471006,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471052,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"16","Value":"522","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"23330790","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3617,"Value":72469,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":20622106,"Value":284700065,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":22,"Value":336,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":11957927,"Value":204812361,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":14,"Value":423,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":14,"Executor Deserialize CPU Time":11957927,"Executor Run Time":22,"Executor CPU Time":20622106,"Result Size":3617,"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":"SparkListenerStageCompleted","Stage Info":{"Stage ID":10,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":20,"RDD Info":[{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"50\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[9],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788470562,"Completion Time":1547788471053,"Accumulables":[{"ID":227,"Name":"aggregate time total (min, med, max)","Value":"25","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Value":336,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":223,"Name":"duration total (min, med, max)","Value":"522","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":318,"Name":"internal.metrics.resultSize","Value":72469,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":225,"Name":"peak memory total (min, med, max)","Value":"23330790","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Value":204812361,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Value":284700065,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":320,"Name":"internal.metrics.resultSerializationTime","Value":3,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Value":423,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":7,"Completion Time":1547788471054,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":8,"Submission Time":1547788471064,"Stage Infos":[{"Stage ID":12,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":100,"RDD Info":[{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"50\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[11],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]},{"Stage ID":11,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":1,"RDD Info":[{"RDD ID":21,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":19,"Name":"FileScanRDD","Scope":"{\"id\":\"46\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"46\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[12,11],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"4","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":12,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":100,"RDD Info":[{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"50\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[11],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788471074,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"4","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":17,"Attempt":0,"Launch Time":1547788471082,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":32,"Index":0,"Attempt":0,"Launch Time":1547788471083,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":33,"Index":1,"Attempt":0,"Launch Time":1547788471120,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":31,"Index":17,"Attempt":0,"Launch Time":1547788471082,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471121,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"528","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":228,"Name":"avg hash probe (min, med, max)","Update":"10","Value":"20","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":224,"Name":"number of output rows","Update":"1","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"17039359","Value":"40370149","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":227,"Name":"aggregate time total (min, med, max)","Update":"0","Value":"23","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":204,"Name":"local blocks read","Update":"1","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":207,"Name":"local bytes read total (min, med, max)","Update":"59","Value":"96","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":209,"Name":"records read","Update":"1","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":60,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":17039360,"Value":17039360,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3643,"Value":3643,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":10444642,"Value":10444642,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":11,"Value":11,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":11614540,"Value":11614540,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":20,"Value":20,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":20,"Executor Deserialize CPU Time":11614540,"Executor Run Time":11,"Executor CPU Time":10444642,"Result Size":3643,"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":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":60,"Total Records Read":1},"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":34,"Index":2,"Attempt":0,"Launch Time":1547788471147,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":33,"Index":1,"Attempt":0,"Launch Time":1547788471120,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471147,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"7","Value":"535","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"40632292","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":17301504,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":7260,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":11458391,"Value":21903033,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":12,"Value":23,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7200707,"Value":18815247,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":27,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":7200707,"Executor Run Time":12,"Executor CPU Time":11458391,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":35,"Index":3,"Attempt":0,"Launch Time":1547788471158,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":32,"Index":0,"Attempt":0,"Launch Time":1547788471083,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471159,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"23","Value":"558","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"40894435","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":17563648,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":10877,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":18955449,"Value":40858482,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":28,"Value":51,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":22943627,"Value":41758874,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":36,"Value":63,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":36,"Executor Deserialize CPU Time":22943627,"Executor Run Time":28,"Executor CPU Time":18955449,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":36,"Index":4,"Attempt":0,"Launch Time":1547788471172,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":34,"Index":2,"Attempt":0,"Launch Time":1547788471147,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471172,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"564","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"41156578","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":17825792,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":14494,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9948802,"Value":50807284,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":11,"Value":62,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6934447,"Value":48693321,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":70,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":6934447,"Executor Run Time":11,"Executor CPU Time":9948802,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":37,"Index":5,"Attempt":0,"Launch Time":1547788471191,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":35,"Index":3,"Attempt":0,"Launch Time":1547788471158,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471192,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"10","Value":"574","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"41418721","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":18087936,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":18111,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":15075464,"Value":65882748,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":15,"Value":77,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7974927,"Value":56668248,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":8,"Value":78,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":8,"Executor Deserialize CPU Time":7974927,"Executor Run Time":15,"Executor CPU Time":15075464,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":38,"Index":6,"Attempt":0,"Launch Time":1547788471192,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":36,"Index":4,"Attempt":0,"Launch Time":1547788471172,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471193,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"580","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"41680864","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":18350080,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":21728,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9061776,"Value":74944524,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":86,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5539711,"Value":62207959,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":84,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5539711,"Executor Run Time":9,"Executor CPU Time":9061776,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":39,"Index":7,"Attempt":0,"Launch Time":1547788471224,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":38,"Index":6,"Attempt":0,"Launch Time":1547788471192,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471225,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"10","Value":"590","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"41943007","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":18612224,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":25345,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":12191130,"Value":87135654,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":15,"Value":101,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7951278,"Value":70159237,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":8,"Value":92,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":8,"Executor Deserialize CPU Time":7951278,"Executor Run Time":15,"Executor CPU Time":12191130,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":40,"Index":8,"Attempt":0,"Launch Time":1547788471238,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":37,"Index":5,"Attempt":0,"Launch Time":1547788471191,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471239,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"13","Value":"603","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"42205150","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":18874368,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":28962,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":14913138,"Value":102048792,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":18,"Value":119,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":10083135,"Value":80242372,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":11,"Value":103,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":11,"Executor Deserialize CPU Time":10083135,"Executor Run Time":18,"Executor CPU Time":14913138,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":41,"Index":9,"Attempt":0,"Launch Time":1547788471251,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":39,"Index":7,"Attempt":0,"Launch Time":1547788471224,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471251,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"7","Value":"610","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"42467293","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":19136512,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":32579,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":11277425,"Value":113326217,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":12,"Value":131,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7366758,"Value":87609130,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":110,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":7366758,"Executor Run Time":12,"Executor CPU Time":11277425,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":42,"Index":10,"Attempt":0,"Launch Time":1547788471272,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":40,"Index":8,"Attempt":0,"Launch Time":1547788471238,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471273,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"10","Value":"620","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"42729436","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":19398656,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":36196,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":13862819,"Value":127189036,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":15,"Value":146,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8801575,"Value":96410705,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":10,"Value":120,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":10,"Executor Deserialize CPU Time":8801575,"Executor Run Time":15,"Executor CPU Time":13862819,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":43,"Index":11,"Attempt":0,"Launch Time":1547788471274,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":41,"Index":9,"Attempt":0,"Launch Time":1547788471251,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471274,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"626","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"42991579","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":19660800,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":39813,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9404567,"Value":136593603,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":10,"Value":156,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5976060,"Value":102386765,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":126,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5976060,"Executor Run Time":10,"Executor CPU Time":9404567,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":44,"Index":12,"Attempt":0,"Launch Time":1547788471298,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":43,"Index":11,"Attempt":0,"Launch Time":1547788471274,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471299,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"631","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"43253722","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":19922944,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":43430,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9384454,"Value":145978057,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":165,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8651421,"Value":111038186,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":9,"Value":135,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":9,"Executor Deserialize CPU Time":8651421,"Executor Run Time":9,"Executor CPU Time":9384454,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":45,"Index":13,"Attempt":0,"Launch Time":1547788471308,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":42,"Index":10,"Attempt":0,"Launch Time":1547788471272,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471308,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"11","Value":"642","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"43515865","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":20185088,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":47047,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":14241163,"Value":160219220,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":16,"Value":181,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8736676,"Value":119774862,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":10,"Value":145,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":10,"Executor Deserialize CPU Time":8736676,"Executor Run Time":16,"Executor CPU Time":14241163,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":46,"Index":14,"Attempt":0,"Launch Time":1547788471320,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":44,"Index":12,"Attempt":0,"Launch Time":1547788471298,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471320,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"647","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"43778008","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":20447232,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":50664,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":8133337,"Value":168352557,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":190,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5486744,"Value":125261606,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":151,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5486744,"Executor Run Time":9,"Executor CPU Time":8133337,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":47,"Index":15,"Attempt":0,"Launch Time":1547788471342,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":46,"Index":14,"Attempt":0,"Launch Time":1547788471320,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471342,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"653","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"44040151","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":20709376,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":54281,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9024908,"Value":177377465,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":10,"Value":200,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4887146,"Value":130148752,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":156,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4887146,"Executor Run Time":10,"Executor CPU Time":9024908,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":48,"Index":16,"Attempt":0,"Launch Time":1547788471348,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":45,"Index":13,"Attempt":0,"Launch Time":1547788471308,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471348,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"12","Value":"665","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"44302294","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":227,"Name":"aggregate time total (min, med, max)","Update":"0","Value":"9","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":20971520,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":57898,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":14226807,"Value":191604272,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":17,"Value":217,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8717459,"Value":138866211,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":12,"Value":168,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":12,"Executor Deserialize CPU Time":8717459,"Executor Run Time":17,"Executor CPU Time":14226807,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":49,"Index":18,"Attempt":0,"Launch Time":1547788471364,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":47,"Index":15,"Attempt":0,"Launch Time":1547788471342,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471364,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"671","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"44564437","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":21233664,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":61515,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9689284,"Value":201293556,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":10,"Value":227,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4958749,"Value":143824960,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":173,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4958749,"Executor Run Time":10,"Executor CPU Time":9689284,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":50,"Index":19,"Attempt":0,"Launch Time":1547788471383,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":48,"Index":16,"Attempt":0,"Launch Time":1547788471348,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471384,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"13","Value":"684","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"44826580","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":21495808,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":65132,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":14828884,"Value":216122440,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":18,"Value":245,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8682222,"Value":152507182,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":9,"Value":182,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":9,"Executor Deserialize CPU Time":8682222,"Executor Run Time":18,"Executor CPU Time":14828884,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":51,"Index":20,"Attempt":0,"Launch Time":1547788471385,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":49,"Index":18,"Attempt":0,"Launch Time":1547788471364,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471386,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"689","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"45088723","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":21757952,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":68749,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":8110068,"Value":224232508,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":8,"Value":253,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6796299,"Value":159303481,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":189,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":6796299,"Executor Run Time":8,"Executor CPU Time":8110068,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":52,"Index":21,"Attempt":0,"Launch Time":1547788471405,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":51,"Index":20,"Attempt":0,"Launch Time":1547788471385,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471406,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"694","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"45350866","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":22020096,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":72366,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":8359516,"Value":232592024,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":262,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5457354,"Value":164760835,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":194,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5457354,"Executor Run Time":9,"Executor CPU Time":8359516,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":53,"Index":22,"Attempt":0,"Launch Time":1547788471412,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":50,"Index":19,"Attempt":0,"Launch Time":1547788471383,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471412,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"7","Value":"701","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"45613009","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":22282240,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":75983,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":11236782,"Value":243828806,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":11,"Value":273,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7990768,"Value":172751603,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":9,"Value":203,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":9,"Executor Deserialize CPU Time":7990768,"Executor Run Time":11,"Executor CPU Time":11236782,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":54,"Index":23,"Attempt":0,"Launch Time":1547788471426,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":52,"Index":21,"Attempt":0,"Launch Time":1547788471405,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471427,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"706","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"45875152","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":22544384,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":79600,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":8553675,"Value":252382481,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":282,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5575600,"Value":178327203,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":208,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5575600,"Executor Run Time":9,"Executor CPU Time":8553675,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":55,"Index":24,"Attempt":0,"Launch Time":1547788471447,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":53,"Index":22,"Attempt":0,"Launch Time":1547788471412,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471448,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"11","Value":"717","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"46137295","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":22806528,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":83217,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":16255120,"Value":268637601,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":18,"Value":300,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":9055711,"Value":187382914,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":10,"Value":218,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":10,"Executor Deserialize CPU Time":9055711,"Executor Run Time":18,"Executor CPU Time":16255120,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":56,"Index":25,"Attempt":0,"Launch Time":1547788471448,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":54,"Index":23,"Attempt":0,"Launch Time":1547788471426,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471449,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"722","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"46399438","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":23068672,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":86834,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":8880916,"Value":277518517,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":309,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5387499,"Value":192770413,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":223,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5387499,"Executor Run Time":9,"Executor CPU Time":8880916,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":57,"Index":26,"Attempt":0,"Launch Time":1547788471469,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":56,"Index":25,"Attempt":0,"Launch Time":1547788471448,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471469,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"727","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"46661581","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":23330816,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":90451,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":8568476,"Value":286086993,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":318,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5414263,"Value":198184676,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":229,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5414263,"Executor Run Time":9,"Executor CPU Time":8568476,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":58,"Index":27,"Attempt":0,"Launch Time":1547788471490,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":57,"Index":26,"Attempt":0,"Launch Time":1547788471469,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471490,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"732","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"46923724","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":23592960,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":94068,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":8628386,"Value":294715379,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":327,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5388011,"Value":203572687,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":234,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5388011,"Executor Run Time":9,"Executor CPU Time":8628386,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":59,"Index":28,"Attempt":0,"Launch Time":1547788471491,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":55,"Index":24,"Attempt":0,"Launch Time":1547788471447,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471491,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"11","Value":"743","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"47185867","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":23855104,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":97685,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":15818993,"Value":310534372,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":16,"Value":343,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":9904112,"Value":213476799,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":18,"Value":252,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":18,"Executor Deserialize CPU Time":9904112,"Executor Run Time":16,"Executor CPU Time":15818993,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":60,"Index":29,"Attempt":0,"Launch Time":1547788471512,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":58,"Index":27,"Attempt":0,"Launch Time":1547788471490,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471513,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"7","Value":"750","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"47448010","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":24117248,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":101302,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":8960508,"Value":319494880,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":10,"Value":353,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5457103,"Value":218933902,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":258,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5457103,"Executor Run Time":10,"Executor CPU Time":8960508,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":61,"Index":30,"Attempt":0,"Launch Time":1547788471532,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":60,"Index":29,"Attempt":0,"Launch Time":1547788471512,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471532,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"755","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"47710153","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":24379392,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":104919,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":7925873,"Value":327420753,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":362,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4665435,"Value":223599337,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":263,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4665435,"Executor Run Time":9,"Executor CPU Time":7925873,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":62,"Index":31,"Attempt":0,"Launch Time":1547788471536,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":59,"Index":28,"Attempt":0,"Launch Time":1547788471491,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471537,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"10","Value":"765","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"47972296","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":24641536,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":108536,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":16624938,"Value":344045691,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":18,"Value":380,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":13182460,"Value":236781797,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":19,"Value":282,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":19,"Executor Deserialize CPU Time":13182460,"Executor Run Time":18,"Executor CPU Time":16624938,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":63,"Index":32,"Attempt":0,"Launch Time":1547788471557,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":61,"Index":30,"Attempt":0,"Launch Time":1547788471532,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471557,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"7","Value":"772","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"48234439","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":24903680,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":112153,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":11065271,"Value":355110962,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":11,"Value":391,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7259375,"Value":244041172,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":289,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":7259375,"Executor Run Time":11,"Executor CPU Time":11065271,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":64,"Index":33,"Attempt":0,"Launch Time":1547788471575,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":63,"Index":32,"Attempt":0,"Launch Time":1547788471557,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471575,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"776","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"48496582","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":25165824,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":115770,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":7210861,"Value":362321823,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":7,"Value":398,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4726627,"Value":248767799,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":294,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4726627,"Executor Run Time":7,"Executor CPU Time":7210861,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":65,"Index":34,"Attempt":0,"Launch Time":1547788471579,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":62,"Index":31,"Attempt":0,"Launch Time":1547788471536,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471580,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"12","Value":"788","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"48758725","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":25427968,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":119387,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":17941566,"Value":380263389,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":18,"Value":416,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":13630205,"Value":262398004,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":14,"Value":308,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":14,"Executor Deserialize CPU Time":13630205,"Executor Run Time":18,"Executor CPU Time":17941566,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":66,"Index":35,"Attempt":0,"Launch Time":1547788471596,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":64,"Index":33,"Attempt":0,"Launch Time":1547788471575,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471597,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"793","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"49020868","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":25690112,"Internal":true,"Count Failed Values":true},{"ID":345,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3660,"Value":123047,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":8912458,"Value":389175847,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":8,"Value":424,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6097744,"Value":268495748,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":315,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":6097744,"Executor Run Time":8,"Executor CPU Time":8912458,"Result Size":3660,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":67,"Index":36,"Attempt":0,"Launch Time":1547788471617,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":65,"Index":34,"Attempt":0,"Launch Time":1547788471579,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471617,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"13","Value":"806","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"49283011","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":25952256,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":126664,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":15199954,"Value":404375801,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":17,"Value":441,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8754809,"Value":277250557,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":11,"Value":326,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":11,"Executor Deserialize CPU Time":8754809,"Executor Run Time":17,"Executor CPU Time":15199954,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":68,"Index":37,"Attempt":0,"Launch Time":1547788471619,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":66,"Index":35,"Attempt":0,"Launch Time":1547788471596,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471620,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"812","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"49545154","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":26214400,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":130281,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9577896,"Value":413953697,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":450,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5853991,"Value":283104548,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":333,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":5853991,"Executor Run Time":9,"Executor CPU Time":9577896,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":69,"Index":38,"Attempt":0,"Launch Time":1547788471639,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":68,"Index":37,"Attempt":0,"Launch Time":1547788471619,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471640,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"816","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"49807297","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":26476544,"Internal":true,"Count Failed Values":true},{"ID":345,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3660,"Value":133941,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":7837850,"Value":421791547,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":8,"Value":458,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4694415,"Value":287798963,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":337,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4694415,"Executor Run Time":8,"Executor CPU Time":7837850,"Result Size":3660,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":70,"Index":39,"Attempt":0,"Launch Time":1547788471648,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":67,"Index":36,"Attempt":0,"Launch Time":1547788471617,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471649,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"10","Value":"826","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"50069440","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":26738688,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":137558,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":14699497,"Value":436491044,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":15,"Value":473,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8375616,"Value":296174579,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":9,"Value":346,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":9,"Executor Deserialize CPU Time":8375616,"Executor Run Time":15,"Executor CPU Time":14699497,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":71,"Index":40,"Attempt":0,"Launch Time":1547788471657,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":69,"Index":38,"Attempt":0,"Launch Time":1547788471639,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471658,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"830","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"50331583","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":27000832,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":141175,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":7034601,"Value":443525645,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":8,"Value":481,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4375578,"Value":300550157,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":350,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4375578,"Executor Run Time":8,"Executor CPU Time":7034601,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":72,"Index":41,"Attempt":0,"Launch Time":1547788471678,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":71,"Index":40,"Attempt":0,"Launch Time":1547788471657,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471678,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"835","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"50593726","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":27262976,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":144792,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":8094897,"Value":451620542,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":490,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5521327,"Value":306071484,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":356,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5521327,"Executor Run Time":9,"Executor CPU Time":8094897,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":73,"Index":42,"Attempt":0,"Launch Time":1547788471688,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":70,"Index":39,"Attempt":0,"Launch Time":1547788471648,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471688,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"11","Value":"846","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"50855869","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":27525120,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":148409,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":15249188,"Value":466869730,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":20,"Value":510,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8547465,"Value":314618949,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":11,"Value":367,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":11,"Executor Deserialize CPU Time":8547465,"Executor Run Time":20,"Executor CPU Time":15249188,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":74,"Index":43,"Attempt":0,"Launch Time":1547788471696,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":72,"Index":41,"Attempt":0,"Launch Time":1547788471678,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471697,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"851","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"51118012","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":27787264,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":152026,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":8058368,"Value":474928098,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":8,"Value":518,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5166115,"Value":319785064,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":372,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5166115,"Executor Run Time":8,"Executor CPU Time":8058368,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":75,"Index":44,"Attempt":0,"Launch Time":1547788471718,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":73,"Index":42,"Attempt":0,"Launch Time":1547788471688,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471719,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"9","Value":"860","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"51380155","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":28049408,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":155643,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":13338594,"Value":488266692,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":14,"Value":532,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8079370,"Value":327864434,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":9,"Value":381,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":9,"Executor Deserialize CPU Time":8079370,"Executor Run Time":14,"Executor CPU Time":13338594,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":76,"Index":45,"Attempt":0,"Launch Time":1547788471720,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":74,"Index":43,"Attempt":0,"Launch Time":1547788471696,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471720,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"7","Value":"867","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"51642298","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":28311552,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":159260,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9956634,"Value":498223326,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":11,"Value":543,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6060346,"Value":333924780,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":387,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":6060346,"Executor Run Time":11,"Executor CPU Time":9956634,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":77,"Index":46,"Attempt":0,"Launch Time":1547788471737,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":76,"Index":45,"Attempt":0,"Launch Time":1547788471720,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471737,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"871","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"51904441","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":28573696,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":162877,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":7143104,"Value":505366430,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":7,"Value":550,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4396192,"Value":338320972,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":391,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4396192,"Executor Run Time":7,"Executor CPU Time":7143104,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":78,"Index":47,"Attempt":0,"Launch Time":1547788471758,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":75,"Index":44,"Attempt":0,"Launch Time":1547788471718,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471759,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"13","Value":"884","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"52166584","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":28835840,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":166494,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":15062982,"Value":520429412,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":21,"Value":571,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7833118,"Value":346154090,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":9,"Value":400,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":9,"Executor Deserialize CPU Time":7833118,"Executor Run Time":21,"Executor CPU Time":15062982,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":79,"Index":48,"Attempt":0,"Launch Time":1547788471759,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":77,"Index":46,"Attempt":0,"Launch Time":1547788471737,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471759,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"7","Value":"891","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"52428727","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":29097984,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":170111,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":10284571,"Value":530713983,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":580,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6158828,"Value":352312918,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":8,"Value":408,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":8,"Executor Deserialize CPU Time":6158828,"Executor Run Time":9,"Executor CPU Time":10284571,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":80,"Index":49,"Attempt":0,"Launch Time":1547788471780,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":79,"Index":48,"Attempt":0,"Launch Time":1547788471759,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471780,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"897","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"52690870","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":29360128,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":173728,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9330053,"Value":540044036,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":589,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5812350,"Value":358125268,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":414,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5812350,"Executor Run Time":9,"Executor CPU Time":9330053,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":81,"Index":50,"Attempt":0,"Launch Time":1547788471801,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":80,"Index":49,"Attempt":0,"Launch Time":1547788471780,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471802,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"903","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"52953013","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":29622272,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":177345,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9508952,"Value":549552988,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":8,"Value":597,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6126710,"Value":364251978,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":421,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":6126710,"Executor Run Time":8,"Executor CPU Time":9508952,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":82,"Index":51,"Attempt":0,"Launch Time":1547788471803,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":78,"Index":47,"Attempt":0,"Launch Time":1547788471758,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471803,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"9","Value":"912","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"53215156","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":29884416,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":180962,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":13325303,"Value":562878291,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":15,"Value":612,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8203334,"Value":372455312,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":9,"Value":430,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":9,"Executor Deserialize CPU Time":8203334,"Executor Run Time":15,"Executor CPU Time":13325303,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":83,"Index":52,"Attempt":0,"Launch Time":1547788471823,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":81,"Index":50,"Attempt":0,"Launch Time":1547788471801,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471823,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"918","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"53477299","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":30146560,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":184579,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9350207,"Value":572228498,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":621,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5957978,"Value":378413290,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":436,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5957978,"Executor Run Time":9,"Executor CPU Time":9350207,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":84,"Index":53,"Attempt":0,"Launch Time":1547788471844,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":83,"Index":52,"Attempt":0,"Launch Time":1547788471823,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471845,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"924","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"53739442","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":30408704,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":188196,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9388960,"Value":581617458,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":630,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6240504,"Value":384653794,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":442,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":6240504,"Executor Run Time":9,"Executor CPU Time":9388960,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":85,"Index":54,"Attempt":0,"Launch Time":1547788471866,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":84,"Index":53,"Attempt":0,"Launch Time":1547788471844,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471866,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"930","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"54001585","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":30670848,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":191813,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9485397,"Value":591102855,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":10,"Value":640,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5969392,"Value":390623186,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":448,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5969392,"Executor Run Time":10,"Executor CPU Time":9485397,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":86,"Index":55,"Attempt":0,"Launch Time":1547788471886,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":85,"Index":54,"Attempt":0,"Launch Time":1547788471866,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471886,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"935","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"54263728","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":30932992,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":195430,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9059824,"Value":600162679,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":8,"Value":648,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5657365,"Value":396280551,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":454,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5657365,"Executor Run Time":8,"Executor CPU Time":9059824,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":87,"Index":56,"Attempt":0,"Launch Time":1547788471898,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":82,"Index":51,"Attempt":0,"Launch Time":1547788471803,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471898,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"26","Value":"961","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"54525871","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":31195136,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":199047,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":15266506,"Value":615429185,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":42,"Value":690,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8688693,"Value":404969244,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":26,"Value":480,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":26,"Executor Deserialize CPU Time":8688693,"Executor Run Time":42,"Executor CPU Time":15266506,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":88,"Index":57,"Attempt":0,"Launch Time":1547788471908,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":86,"Index":55,"Attempt":0,"Launch Time":1547788471886,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471908,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"967","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"54788014","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":31457280,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":202664,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9852648,"Value":625281833,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":10,"Value":700,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5871602,"Value":410840846,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":486,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5871602,"Executor Run Time":10,"Executor CPU Time":9852648,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":89,"Index":58,"Attempt":0,"Launch Time":1547788471933,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":88,"Index":57,"Attempt":0,"Launch Time":1547788471908,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471934,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"973","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"55050157","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":31719424,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":206281,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9435784,"Value":634717617,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":709,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6260038,"Value":417100884,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":492,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":6260038,"Executor Run Time":9,"Executor CPU Time":9435784,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":90,"Index":59,"Attempt":0,"Launch Time":1547788471955,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":89,"Index":58,"Attempt":0,"Launch Time":1547788471933,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471955,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"978","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"55312300","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":31981568,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":209898,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9084482,"Value":643802099,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":8,"Value":717,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5882081,"Value":422982965,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":498,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5882081,"Executor Run Time":8,"Executor CPU Time":9084482,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":91,"Index":60,"Attempt":0,"Launch Time":1547788471965,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":87,"Index":56,"Attempt":0,"Launch Time":1547788471898,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471965,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"29","Value":"1007","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"55574443","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":227,"Name":"aggregate time total (min, med, max)","Update":"2","Value":"-31","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":32243712,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":213515,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":16055581,"Value":659857680,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":37,"Value":754,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8515827,"Value":431498792,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":17,"Value":515,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":17,"Executor Deserialize CPU Time":8515827,"Executor Run Time":37,"Executor CPU Time":16055581,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":92,"Index":61,"Attempt":0,"Launch Time":1547788471976,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":90,"Index":59,"Attempt":0,"Launch Time":1547788471955,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471976,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"1013","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"55836586","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":32505856,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":217132,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9697936,"Value":669555616,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":763,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5797880,"Value":437296672,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":521,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5797880,"Executor Run Time":9,"Executor CPU Time":9697936,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":93,"Index":62,"Attempt":0,"Launch Time":1547788471995,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":92,"Index":61,"Attempt":0,"Launch Time":1547788471976,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471995,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1018","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"56098729","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":32768000,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":220749,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":8560813,"Value":678116429,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":772,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5638871,"Value":442935543,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":527,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5638871,"Executor Run Time":9,"Executor CPU Time":8560813,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":94,"Index":63,"Attempt":0,"Launch Time":1547788472000,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":91,"Index":60,"Attempt":0,"Launch Time":1547788471965,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472001,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"17","Value":"1035","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"56360872","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":33030144,"Internal":true,"Count Failed Values":true},{"ID":345,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3660,"Value":224409,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":12009815,"Value":690126244,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":21,"Value":793,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6583189,"Value":449518732,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":8,"Value":535,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":8,"Executor Deserialize CPU Time":6583189,"Executor Run Time":21,"Executor CPU Time":12009815,"Result Size":3660,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":95,"Index":64,"Attempt":0,"Launch Time":1547788472015,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":93,"Index":62,"Attempt":0,"Launch Time":1547788471995,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472015,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"1041","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"56623015","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":33292288,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":228026,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":8550049,"Value":698676293,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":802,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5520058,"Value":455038790,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":540,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5520058,"Executor Run Time":9,"Executor CPU Time":8550049,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":96,"Index":65,"Attempt":0,"Launch Time":1547788472060,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":94,"Index":63,"Attempt":0,"Launch Time":1547788472000,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472061,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"37","Value":"1078","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"56885158","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":33554432,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":231643,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":13494410,"Value":712170703,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":41,"Value":843,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6619804,"Value":461658594,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":547,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":6619804,"Executor Run Time":41,"Executor CPU Time":13494410,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":97,"Index":66,"Attempt":0,"Launch Time":1547788472107,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":95,"Index":64,"Attempt":0,"Launch Time":1547788472015,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472108,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"76","Value":"1154","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"57147301","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":33816576,"Internal":true,"Count Failed Values":true},{"ID":344,"Name":"internal.metrics.jvmGCTime","Update":68,"Value":68,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3660,"Value":235303,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":10897540,"Value":723068243,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":80,"Value":923,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6397648,"Value":468056242,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":553,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":6397648,"Executor Run Time":80,"Executor CPU Time":10897540,"Result Size":3660,"JVM GC Time":68,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":98,"Index":67,"Attempt":0,"Launch Time":1547788472129,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":97,"Index":66,"Attempt":0,"Launch Time":1547788472107,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472130,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1158","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"57409444","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":34078720,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":238920,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":8256923,"Value":731325166,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":10,"Value":933,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6123808,"Value":474180050,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":559,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":6123808,"Executor Run Time":10,"Executor CPU Time":8256923,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":99,"Index":68,"Attempt":0,"Launch Time":1547788472144,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":96,"Index":65,"Attempt":0,"Launch Time":1547788472060,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472144,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"48","Value":"1206","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"57671587","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":34340864,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":242537,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":14766579,"Value":746091745,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":58,"Value":991,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7033307,"Value":481213357,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":17,"Value":576,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":17,"Executor Deserialize CPU Time":7033307,"Executor Run Time":58,"Executor CPU Time":14766579,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":100,"Index":69,"Attempt":0,"Launch Time":1547788472146,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":98,"Index":67,"Attempt":0,"Launch Time":1547788472129,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472146,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1210","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"57933730","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":34603008,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":246154,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":7143208,"Value":753234953,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":8,"Value":999,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4411266,"Value":485624623,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":580,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4411266,"Executor Run Time":8,"Executor CPU Time":7143208,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":101,"Index":70,"Attempt":0,"Launch Time":1547788472163,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":100,"Index":69,"Attempt":0,"Launch Time":1547788472146,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472164,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1214","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"58195873","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":34865152,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":249771,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":6949656,"Value":760184609,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":6,"Value":1005,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4403849,"Value":490028472,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":585,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4403849,"Executor Run Time":6,"Executor CPU Time":6949656,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":102,"Index":71,"Attempt":0,"Launch Time":1547788472178,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":99,"Index":68,"Attempt":0,"Launch Time":1547788472144,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472179,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"9","Value":"1223","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"58458016","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":35127296,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":253388,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":12131790,"Value":772316399,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":13,"Value":1018,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7962672,"Value":497991144,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":11,"Value":596,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":11,"Executor Deserialize CPU Time":7962672,"Executor Run Time":13,"Executor CPU Time":12131790,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":103,"Index":72,"Attempt":0,"Launch Time":1547788472180,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":101,"Index":70,"Attempt":0,"Launch Time":1547788472163,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472180,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1227","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"58720159","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":35389440,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":257005,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":6978432,"Value":779294831,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":8,"Value":1026,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4476822,"Value":502467966,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":600,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4476822,"Executor Run Time":8,"Executor CPU Time":6978432,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":104,"Index":73,"Attempt":0,"Launch Time":1547788472196,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":103,"Index":72,"Attempt":0,"Launch Time":1547788472180,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472196,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1231","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"58982302","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":35651584,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":260622,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":6807151,"Value":786101982,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":8,"Value":1034,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4341850,"Value":506809816,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":603,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":4341850,"Executor Run Time":8,"Executor CPU Time":6807151,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":105,"Index":74,"Attempt":0,"Launch Time":1547788472204,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":102,"Index":71,"Attempt":0,"Launch Time":1547788472178,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472204,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"7","Value":"1238","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"59244445","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":35913728,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":264239,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":10728505,"Value":796830487,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":12,"Value":1046,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6101014,"Value":512910830,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":609,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":6101014,"Executor Run Time":12,"Executor CPU Time":10728505,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":106,"Index":75,"Attempt":0,"Launch Time":1547788472211,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":104,"Index":73,"Attempt":0,"Launch Time":1547788472196,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472212,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1241","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"59506588","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":36175872,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":267856,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":6550817,"Value":803381304,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":7,"Value":1053,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4359486,"Value":517270316,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":613,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4359486,"Executor Run Time":7,"Executor CPU Time":6550817,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":107,"Index":76,"Attempt":0,"Launch Time":1547788472224,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":105,"Index":74,"Attempt":0,"Launch Time":1547788472204,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472225,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"1247","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"59768731","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":36438016,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":271473,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9032531,"Value":812413835,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":1062,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4621545,"Value":521891861,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":618,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4621545,"Executor Run Time":9,"Executor CPU Time":9032531,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":108,"Index":77,"Attempt":0,"Launch Time":1547788472228,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":106,"Index":75,"Attempt":0,"Launch Time":1547788472211,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472228,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1250","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"60030874","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":36700160,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":275090,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":6409712,"Value":818823547,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":6,"Value":1068,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4324233,"Value":526216094,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":623,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4324233,"Executor Run Time":6,"Executor CPU Time":6409712,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":109,"Index":78,"Attempt":0,"Launch Time":1547788472243,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":108,"Index":77,"Attempt":0,"Launch Time":1547788472228,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472243,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1253","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"60293017","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":36962304,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":278707,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":5948893,"Value":824772440,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":7,"Value":1075,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4157099,"Value":530373193,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":626,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":4157099,"Executor Run Time":7,"Executor CPU Time":5948893,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":110,"Index":79,"Attempt":0,"Launch Time":1547788472255,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":107,"Index":76,"Attempt":0,"Launch Time":1547788472224,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472256,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"9","Value":"1262","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"60555160","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":227,"Name":"aggregate time total (min, med, max)","Update":"0","Value":"-49","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":37224448,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":282324,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":12000030,"Value":836772470,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":13,"Value":1088,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6168677,"Value":536541870,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":633,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":6168677,"Executor Run Time":13,"Executor CPU Time":12000030,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":111,"Index":80,"Attempt":0,"Launch Time":1547788472260,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":109,"Index":78,"Attempt":0,"Launch Time":1547788472243,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472260,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1265","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"60817303","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":37486592,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":285941,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":6510742,"Value":843283212,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":7,"Value":1095,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4303260,"Value":540845130,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":636,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":4303260,"Executor Run Time":7,"Executor CPU Time":6510742,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":112,"Index":81,"Attempt":0,"Launch Time":1547788472276,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":111,"Index":80,"Attempt":0,"Launch Time":1547788472260,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472277,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1269","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"61079446","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":37748736,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":289558,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":6908036,"Value":850191248,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":6,"Value":1101,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4403845,"Value":545248975,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":641,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4403845,"Executor Run Time":6,"Executor CPU Time":6908036,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":113,"Index":82,"Attempt":0,"Launch Time":1547788472286,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":110,"Index":79,"Attempt":0,"Launch Time":1547788472255,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472286,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"9","Value":"1278","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"61341589","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":38010880,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":293175,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":12129290,"Value":862320538,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":14,"Value":1115,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7920563,"Value":553169538,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":8,"Value":649,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":8,"Executor Deserialize CPU Time":7920563,"Executor Run Time":14,"Executor CPU Time":12129290,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":114,"Index":83,"Attempt":0,"Launch Time":1547788472291,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":112,"Index":81,"Attempt":0,"Launch Time":1547788472276,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472291,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1281","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"61603732","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":38273024,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":296792,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":6066655,"Value":868387193,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":6,"Value":1121,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3671418,"Value":556840956,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":653,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3671418,"Executor Run Time":6,"Executor CPU Time":6066655,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":115,"Index":84,"Attempt":0,"Launch Time":1547788472305,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":114,"Index":83,"Attempt":0,"Launch Time":1547788472291,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472306,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1284","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"61865875","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":38535168,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":300409,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":5658380,"Value":874045573,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":6,"Value":1127,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3797079,"Value":560638035,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":656,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3797079,"Executor Run Time":6,"Executor CPU Time":5658380,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":116,"Index":85,"Attempt":0,"Launch Time":1547788472310,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":113,"Index":82,"Attempt":0,"Launch Time":1547788472286,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472311,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"1290","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"62128018","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":38797312,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":304026,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":10268198,"Value":884313771,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":10,"Value":1137,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6569994,"Value":567208029,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":9,"Value":665,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":9,"Executor Deserialize CPU Time":6569994,"Executor Run Time":10,"Executor CPU Time":10268198,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":117,"Index":86,"Attempt":0,"Launch Time":1547788472323,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":115,"Index":84,"Attempt":0,"Launch Time":1547788472305,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472324,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1293","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"62390161","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":39059456,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":307643,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":6761333,"Value":891075104,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":7,"Value":1144,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4522127,"Value":571730156,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":670,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4522127,"Executor Run Time":7,"Executor CPU Time":6761333,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":118,"Index":87,"Attempt":0,"Launch Time":1547788472341,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":117,"Index":86,"Attempt":0,"Launch Time":1547788472323,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472342,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1297","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"62652304","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":39321600,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":311260,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":7963864,"Value":899038968,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":8,"Value":1152,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5032419,"Value":576762575,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":675,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5032419,"Executor Run Time":8,"Executor CPU Time":7963864,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":119,"Index":88,"Attempt":0,"Launch Time":1547788472359,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":116,"Index":85,"Attempt":0,"Launch Time":1547788472310,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472360,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"23","Value":"1320","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"62914447","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":39583744,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":314877,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":10779655,"Value":909818623,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":29,"Value":1181,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6727712,"Value":583490287,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":13,"Value":688,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":13,"Executor Deserialize CPU Time":6727712,"Executor Run Time":29,"Executor CPU Time":10779655,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":120,"Index":89,"Attempt":0,"Launch Time":1547788472363,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":118,"Index":87,"Attempt":0,"Launch Time":1547788472341,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472363,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1325","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"63176590","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":39845888,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":318494,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":7516058,"Value":917334681,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":1190,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4921440,"Value":588411727,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":692,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4921440,"Executor Run Time":9,"Executor CPU Time":7516058,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":121,"Index":90,"Attempt":0,"Launch Time":1547788472381,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":120,"Index":89,"Attempt":0,"Launch Time":1547788472363,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472382,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1329","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"63438733","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":40108032,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":322111,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":7885694,"Value":925220375,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":8,"Value":1198,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4935183,"Value":593346910,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":697,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4935183,"Executor Run Time":8,"Executor CPU Time":7885694,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":122,"Index":91,"Attempt":0,"Launch Time":1547788472382,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":119,"Index":88,"Attempt":0,"Launch Time":1547788472359,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472383,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1333","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"63700876","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":40370176,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":325728,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":8096868,"Value":933317243,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":8,"Value":1206,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5212304,"Value":598559214,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":703,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5212304,"Executor Run Time":8,"Executor CPU Time":8096868,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":123,"Index":92,"Attempt":0,"Launch Time":1547788472403,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":121,"Index":90,"Attempt":0,"Launch Time":1547788472381,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472404,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1337","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"63963019","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":40632320,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":329345,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":7567228,"Value":940884471,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":8,"Value":1214,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5508158,"Value":604067372,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":9,"Value":712,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":9,"Executor Deserialize CPU Time":5508158,"Executor Run Time":8,"Executor CPU Time":7567228,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":124,"Index":93,"Attempt":0,"Launch Time":1547788472409,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":122,"Index":91,"Attempt":0,"Launch Time":1547788472382,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472409,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"8","Value":"1345","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"64225162","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":40894464,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":332962,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":10648027,"Value":951532498,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":12,"Value":1226,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5917859,"Value":609985231,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":8,"Value":720,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":8,"Executor Deserialize CPU Time":5917859,"Executor Run Time":12,"Executor CPU Time":10648027,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":125,"Index":94,"Attempt":0,"Launch Time":1547788472419,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":123,"Index":92,"Attempt":0,"Launch Time":1547788472403,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472419,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1349","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"64487305","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":41156608,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":336579,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":6875526,"Value":958408024,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":8,"Value":1234,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3210347,"Value":613195578,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":723,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3210347,"Executor Run Time":8,"Executor CPU Time":6875526,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":126,"Index":95,"Attempt":0,"Launch Time":1547788472438,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":124,"Index":93,"Attempt":0,"Launch Time":1547788472409,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472438,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"1355","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"64749448","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":41418752,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":340196,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9613993,"Value":968022017,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":12,"Value":1246,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6005946,"Value":619201524,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":729,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":6005946,"Executor Run Time":12,"Executor CPU Time":9613993,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":127,"Index":96,"Attempt":0,"Launch Time":1547788472440,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":125,"Index":94,"Attempt":0,"Launch Time":1547788472419,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472441,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"1361","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"65011591","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":41680896,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":343813,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":8756332,"Value":976778349,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":10,"Value":1256,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5992726,"Value":625194250,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":735,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5992726,"Executor Run Time":10,"Executor CPU Time":8756332,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":128,"Index":97,"Attempt":0,"Launch Time":1547788472457,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":127,"Index":96,"Attempt":0,"Launch Time":1547788472440,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472458,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1365","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"65273734","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":41943040,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":347430,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":7036673,"Value":983815022,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":7,"Value":1263,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3641036,"Value":628835286,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":738,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3641036,"Executor Run Time":7,"Executor CPU Time":7036673,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":129,"Index":98,"Attempt":0,"Launch Time":1547788472460,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":126,"Index":95,"Attempt":0,"Launch Time":1547788472438,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472461,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1370","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"65535877","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":42205184,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":351047,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":8455047,"Value":992270069,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":10,"Value":1273,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6254404,"Value":635089690,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":744,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":6254404,"Executor Run Time":10,"Executor CPU Time":8455047,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":130,"Index":99,"Attempt":0,"Launch Time":1547788472474,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":128,"Index":97,"Attempt":0,"Launch Time":1547788472457,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472474,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1374","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"65798020","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":42467328,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":354664,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":7315691,"Value":999585760,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":6,"Value":1279,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3708854,"Value":638798544,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":749,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":3708854,"Executor Run Time":6,"Executor CPU Time":7315691,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":129,"Index":98,"Attempt":0,"Launch Time":1547788472460,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472488,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"9","Value":"1383","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"66060163","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":42729472,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":358281,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9576011,"Value":1009161771,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":13,"Value":1292,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5162911,"Value":643961455,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":8,"Value":757,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":8,"Executor Deserialize CPU Time":5162911,"Executor Run Time":13,"Executor CPU Time":9576011,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":130,"Index":99,"Attempt":0,"Launch Time":1547788472474,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472489,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1386","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"66322306","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":42991616,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":361898,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":5697446,"Value":1014859217,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":6,"Value":1298,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3959798,"Value":647921253,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":761,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3959798,"Executor Run Time":6,"Executor CPU Time":5697446,"Result Size":3617,"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":"SparkListenerStageCompleted","Stage Info":{"Stage ID":12,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":100,"RDD Info":[{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"50\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[11],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788471074,"Completion Time":1547788472490,"Accumulables":[{"ID":227,"Name":"aggregate time total (min, med, max)","Value":"-49","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":209,"Name":"records read","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Value":647921253,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Value":361898,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":223,"Name":"duration total (min, med, max)","Value":"1386","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":345,"Name":"internal.metrics.resultSerializationTime","Value":3,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":60,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Value":42991616,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Value":761,"Internal":true,"Count Failed Values":true},{"ID":225,"Name":"peak memory total (min, med, max)","Value":"66322306","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":342,"Name":"internal.metrics.executorCpuTime","Value":1014859217,"Internal":true,"Count Failed Values":true},{"ID":207,"Name":"local bytes read total (min, med, max)","Value":"96","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":228,"Name":"avg hash probe (min, med, max)","Value":"20","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":204,"Name":"local blocks read","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":341,"Name":"internal.metrics.executorRunTime","Value":1298,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":344,"Name":"internal.metrics.jvmGCTime","Value":68,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":224,"Name":"number of output rows","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"}]}} +{"Event":"SparkListenerJobEnd","Job ID":8,"Completion Time":1547788472491,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":9,"Submission Time":1547788472506,"Stage Infos":[{"Stage ID":13,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":1,"RDD Info":[{"RDD ID":21,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":19,"Name":"FileScanRDD","Scope":"{\"id\":\"46\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"46\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]},{"Stage ID":14,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":75,"RDD Info":[{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"50\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[13],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[13,14],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"4","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":14,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":75,"RDD Info":[{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"50\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[13],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788472512,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"4","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerTaskStart","Stage ID":14,"Stage Attempt ID":0,"Task Info":{"Task ID":131,"Index":43,"Attempt":0,"Launch Time":1547788472520,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":14,"Stage Attempt ID":0,"Task Info":{"Task ID":132,"Index":0,"Attempt":0,"Launch Time":1547788472521,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":14,"Stage Attempt ID":0,"Task Info":{"Task ID":133,"Index":1,"Attempt":0,"Launch Time":1547788472557,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":131,"Index":43,"Attempt":0,"Launch Time":1547788472520,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472558,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"11","Value":"1397","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":228,"Name":"avg hash probe (min, med, max)","Update":"10","Value":"30","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":224,"Name":"number of output rows","Update":"1","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"17039359","Value":"83361665","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":227,"Name":"aggregate time total (min, med, max)","Update":"8","Value":"-63","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":204,"Name":"local blocks read","Update":"1","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":207,"Name":"local bytes read total (min, med, max)","Update":"62","Value":"59","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":209,"Name":"records read","Update":"1","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":63,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":17039360,"Value":17039360,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3656,"Value":3656,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":15381200,"Value":15381200,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":16,"Value":16,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8872982,"Value":8872982,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":15,"Value":15,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":15,"Executor Deserialize CPU Time":8872982,"Executor Run Time":16,"Executor CPU Time":15381200,"Result Size":3656,"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":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":63,"Total Records Read":1},"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":134,"Index":2,"Attempt":0,"Launch Time":1547788472568,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":132,"Index":0,"Attempt":0,"Launch Time":1547788472521,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472568,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1402","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"83623808","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":17301504,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":7273,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":8909017,"Value":24290217,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":9,"Value":25,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":16079042,"Value":24952024,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":32,"Value":47,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":32,"Executor Deserialize CPU Time":16079042,"Executor Run Time":9,"Executor CPU Time":8909017,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":135,"Index":3,"Attempt":0,"Launch Time":1547788472571,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":133,"Index":1,"Attempt":0,"Launch Time":1547788472557,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472571,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1405","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"83885951","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":17563648,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":10890,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5219491,"Value":29509708,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":5,"Value":30,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3278056,"Value":28230080,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":51,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3278056,"Executor Run Time":5,"Executor CPU Time":5219491,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":136,"Index":4,"Attempt":0,"Launch Time":1547788472584,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":135,"Index":3,"Attempt":0,"Launch Time":1547788472571,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472585,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1408","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"84148094","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":17825792,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":14507,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":6013191,"Value":35522899,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":6,"Value":36,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3216254,"Value":31446334,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":54,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3216254,"Executor Run Time":6,"Executor CPU Time":6013191,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":137,"Index":5,"Attempt":0,"Launch Time":1547788472598,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":136,"Index":4,"Attempt":0,"Launch Time":1547788472584,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472598,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1411","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"84410237","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":18087936,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":18124,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5480342,"Value":41003241,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":5,"Value":41,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3527925,"Value":34974259,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":58,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3527925,"Executor Run Time":5,"Executor CPU Time":5480342,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":138,"Index":6,"Attempt":0,"Launch Time":1547788472600,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":134,"Index":2,"Attempt":0,"Launch Time":1547788472568,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472601,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"1417","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"84672380","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":18350080,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":21741,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":9627469,"Value":50630710,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":10,"Value":51,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5302230,"Value":40276489,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":8,"Value":66,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":8,"Executor Deserialize CPU Time":5302230,"Executor Run Time":10,"Executor CPU Time":9627469,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":139,"Index":7,"Attempt":0,"Launch Time":1547788472611,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":137,"Index":5,"Attempt":0,"Launch Time":1547788472598,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472612,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"2","Value":"1419","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"84934523","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":18612224,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":25358,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5276321,"Value":55907031,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":5,"Value":56,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3785216,"Value":44061705,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":70,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3785216,"Executor Run Time":5,"Executor CPU Time":5276321,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":140,"Index":8,"Attempt":0,"Launch Time":1547788472626,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":139,"Index":7,"Attempt":0,"Launch Time":1547788472611,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472627,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1422","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"85196666","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":18874368,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":28975,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":6176643,"Value":62083674,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":6,"Value":62,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3761399,"Value":47823104,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":74,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3761399,"Executor Run Time":6,"Executor CPU Time":6176643,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":141,"Index":9,"Attempt":0,"Launch Time":1547788472641,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":138,"Index":6,"Attempt":0,"Launch Time":1547788472600,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472642,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"25","Value":"1447","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"85458809","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":227,"Name":"aggregate time total (min, med, max)","Update":"4","Value":"-66","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":19136512,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":32592,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":12602772,"Value":74686446,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":28,"Value":90,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4746241,"Value":52569345,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":80,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":4746241,"Executor Run Time":28,"Executor CPU Time":12602772,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":142,"Index":10,"Attempt":0,"Launch Time":1547788472644,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":140,"Index":8,"Attempt":0,"Launch Time":1547788472626,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472644,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1451","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"85720952","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":19398656,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":36209,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7771104,"Value":82457550,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":8,"Value":98,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4715570,"Value":57284915,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":85,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4715570,"Executor Run Time":8,"Executor CPU Time":7771104,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":143,"Index":11,"Attempt":0,"Launch Time":1547788472657,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":142,"Index":10,"Attempt":0,"Launch Time":1547788472644,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472657,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"2","Value":"1453","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"85983095","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":19660800,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":39826,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5452707,"Value":87910257,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":6,"Value":104,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3625734,"Value":60910649,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":88,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3625734,"Executor Run Time":6,"Executor CPU Time":5452707,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":144,"Index":12,"Attempt":0,"Launch Time":1547788472671,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":143,"Index":11,"Attempt":0,"Launch Time":1547788472657,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472671,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"2","Value":"1455","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"86245238","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":19922944,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":43443,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5300134,"Value":93210391,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":6,"Value":110,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3612247,"Value":64522896,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":91,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3612247,"Executor Run Time":6,"Executor CPU Time":5300134,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":145,"Index":13,"Attempt":0,"Launch Time":1547788472678,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":141,"Index":9,"Attempt":0,"Launch Time":1547788472641,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472678,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"12","Value":"1467","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"86507381","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":20185088,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":47060,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":10008577,"Value":103218968,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":18,"Value":128,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6689362,"Value":71212258,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":98,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":6689362,"Executor Run Time":18,"Executor CPU Time":10008577,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":146,"Index":14,"Attempt":0,"Launch Time":1547788472684,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":144,"Index":12,"Attempt":0,"Launch Time":1547788472671,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472684,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"2","Value":"1469","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"86769524","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":20447232,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":50677,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5118993,"Value":108337961,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":5,"Value":133,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3530194,"Value":74742452,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":102,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3530194,"Executor Run Time":5,"Executor CPU Time":5118993,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":147,"Index":15,"Attempt":0,"Launch Time":1547788472696,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":145,"Index":13,"Attempt":0,"Launch Time":1547788472678,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472696,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1474","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"87031667","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":20709376,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":54294,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7866688,"Value":116204649,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":8,"Value":141,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4784844,"Value":79527296,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":107,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4784844,"Executor Run Time":8,"Executor CPU Time":7866688,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":148,"Index":16,"Attempt":0,"Launch Time":1547788472699,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":146,"Index":14,"Attempt":0,"Launch Time":1547788472684,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472699,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"2","Value":"1476","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"87293810","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":20971520,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":57911,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5552266,"Value":121756915,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":5,"Value":146,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4023055,"Value":83550351,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":112,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4023055,"Executor Run Time":5,"Executor CPU Time":5552266,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":149,"Index":17,"Attempt":0,"Launch Time":1547788472712,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":148,"Index":16,"Attempt":0,"Launch Time":1547788472699,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472713,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"2","Value":"1478","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"87555953","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":21233664,"Internal":true,"Count Failed Values":true},{"ID":370,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3660,"Value":61571,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5358929,"Value":127115844,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":5,"Value":151,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3396464,"Value":86946815,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":115,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3396464,"Executor Run Time":5,"Executor CPU Time":5358929,"Result Size":3660,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":150,"Index":18,"Attempt":0,"Launch Time":1547788472716,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":147,"Index":15,"Attempt":0,"Launch Time":1547788472696,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472716,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1483","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"87818096","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":21495808,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":65188,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":8292843,"Value":135408687,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":9,"Value":160,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5069544,"Value":92016359,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":121,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5069544,"Executor Run Time":9,"Executor CPU Time":8292843,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":151,"Index":19,"Attempt":0,"Launch Time":1547788472726,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":149,"Index":17,"Attempt":0,"Launch Time":1547788472712,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472727,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1486","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"88080239","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":21757952,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":68805,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5812644,"Value":141221331,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":6,"Value":166,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4242098,"Value":96258457,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":125,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4242098,"Executor Run Time":6,"Executor CPU Time":5812644,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":152,"Index":20,"Attempt":0,"Launch Time":1547788472732,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":150,"Index":18,"Attempt":0,"Launch Time":1547788472716,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472732,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1490","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"88342382","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":22020096,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":72422,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":6952671,"Value":148174002,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":8,"Value":174,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4692829,"Value":100951286,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":129,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4692829,"Executor Run Time":8,"Executor CPU Time":6952671,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":153,"Index":21,"Attempt":0,"Launch Time":1547788472745,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":151,"Index":19,"Attempt":0,"Launch Time":1547788472726,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472745,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1494","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"88604525","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":22282240,"Internal":true,"Count Failed Values":true},{"ID":370,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3660,"Value":76082,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7842932,"Value":156016934,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":8,"Value":182,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5765915,"Value":106717201,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":134,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5765915,"Executor Run Time":8,"Executor CPU Time":7842932,"Result Size":3660,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":154,"Index":22,"Attempt":0,"Launch Time":1547788472749,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":152,"Index":20,"Attempt":0,"Launch Time":1547788472732,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472750,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1498","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"88866668","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":22544384,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":79699,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7300900,"Value":163317834,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":7,"Value":189,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4355555,"Value":111072756,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":139,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4355555,"Executor Run Time":7,"Executor CPU Time":7300900,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":155,"Index":23,"Attempt":0,"Launch Time":1547788472762,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":153,"Index":21,"Attempt":0,"Launch Time":1547788472745,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472763,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1502","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"89128811","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":22806528,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":83316,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7418972,"Value":170736806,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":8,"Value":197,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5372143,"Value":116444899,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":144,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5372143,"Executor Run Time":8,"Executor CPU Time":7418972,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":156,"Index":24,"Attempt":0,"Launch Time":1547788472768,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":154,"Index":22,"Attempt":0,"Launch Time":1547788472749,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472769,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1506","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"89390954","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":23068672,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":86933,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7366926,"Value":178103732,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":9,"Value":206,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5480706,"Value":121925605,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":149,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5480706,"Executor Run Time":9,"Executor CPU Time":7366926,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":157,"Index":25,"Attempt":0,"Launch Time":1547788472777,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":155,"Index":23,"Attempt":0,"Launch Time":1547788472762,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472778,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"2","Value":"1508","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"89653097","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":23330816,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":90550,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5710660,"Value":183814392,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":6,"Value":212,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4584705,"Value":126510310,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":153,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4584705,"Executor Run Time":6,"Executor CPU Time":5710660,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":158,"Index":26,"Attempt":0,"Launch Time":1547788472791,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":157,"Index":25,"Attempt":0,"Launch Time":1547788472777,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472792,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1511","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"89915240","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":23592960,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":94167,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5788208,"Value":189602600,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":5,"Value":217,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3542437,"Value":130052747,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":157,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3542437,"Executor Run Time":5,"Executor CPU Time":5788208,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":159,"Index":27,"Attempt":0,"Launch Time":1547788472792,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":156,"Index":24,"Attempt":0,"Launch Time":1547788472768,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472793,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"1517","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"90177383","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":23855104,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":97784,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":9339029,"Value":198941629,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":10,"Value":227,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5172685,"Value":135225432,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":164,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":5172685,"Executor Run Time":10,"Executor CPU Time":9339029,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":160,"Index":28,"Attempt":0,"Launch Time":1547788472806,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":158,"Index":26,"Attempt":0,"Launch Time":1547788472791,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472807,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1520","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"90439526","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":24117248,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":101401,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5736880,"Value":204678509,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":6,"Value":233,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3653261,"Value":138878693,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":168,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3653261,"Executor Run Time":6,"Executor CPU Time":5736880,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":161,"Index":29,"Attempt":0,"Launch Time":1547788472815,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":159,"Index":27,"Attempt":0,"Launch Time":1547788472792,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472815,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1525","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"90701669","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":24379392,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":105018,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7310374,"Value":211988883,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":12,"Value":245,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5092058,"Value":143970751,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":173,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5092058,"Executor Run Time":12,"Executor CPU Time":7310374,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":162,"Index":30,"Attempt":0,"Launch Time":1547788472820,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":160,"Index":28,"Attempt":0,"Launch Time":1547788472806,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472821,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"2","Value":"1527","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"90963812","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":24641536,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":108635,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5302259,"Value":217291142,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":5,"Value":250,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3681762,"Value":147652513,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":177,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3681762,"Executor Run Time":5,"Executor CPU Time":5302259,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":163,"Index":31,"Attempt":0,"Launch Time":1547788472834,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":162,"Index":30,"Attempt":0,"Launch Time":1547788472820,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472835,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1530","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"91225955","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":24903680,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":112252,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5629506,"Value":222920648,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":5,"Value":255,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3404257,"Value":151056770,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":181,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3404257,"Executor Run Time":5,"Executor CPU Time":5629506,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":164,"Index":32,"Attempt":0,"Launch Time":1547788472839,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":161,"Index":29,"Attempt":0,"Launch Time":1547788472815,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472839,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1534","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"91488098","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":25165824,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":115869,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7223660,"Value":230144308,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":7,"Value":262,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4732379,"Value":155789149,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":8,"Value":189,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":8,"Executor Deserialize CPU Time":4732379,"Executor Run Time":7,"Executor CPU Time":7223660,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":165,"Index":33,"Attempt":0,"Launch Time":1547788472848,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":163,"Index":31,"Attempt":0,"Launch Time":1547788472834,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472849,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1537","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"91750241","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":25427968,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":119486,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5762476,"Value":235906784,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":6,"Value":268,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3458234,"Value":159247383,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":192,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3458234,"Executor Run Time":6,"Executor CPU Time":5762476,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":166,"Index":34,"Attempt":0,"Launch Time":1547788472858,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":164,"Index":32,"Attempt":0,"Launch Time":1547788472839,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472859,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1541","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"92012384","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":25690112,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":123103,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7748774,"Value":243655558,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":8,"Value":276,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5373685,"Value":164621068,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":198,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5373685,"Executor Run Time":8,"Executor CPU Time":7748774,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":167,"Index":35,"Attempt":0,"Launch Time":1547788472863,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":165,"Index":33,"Attempt":0,"Launch Time":1547788472848,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472863,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1544","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"92274527","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":25952256,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":126720,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5611576,"Value":249267134,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":5,"Value":281,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3702884,"Value":168323952,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":202,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3702884,"Executor Run Time":5,"Executor CPU Time":5611576,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":168,"Index":36,"Attempt":0,"Launch Time":1547788472877,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":167,"Index":35,"Attempt":0,"Launch Time":1547788472863,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472878,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1547","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"92536670","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":26214400,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":130337,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":6093576,"Value":255360710,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":6,"Value":287,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3726766,"Value":172050718,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":205,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3726766,"Executor Run Time":6,"Executor CPU Time":6093576,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":169,"Index":37,"Attempt":0,"Launch Time":1547788472878,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":166,"Index":34,"Attempt":0,"Launch Time":1547788472858,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472878,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1552","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"92798813","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":26476544,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":133954,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7716743,"Value":263077453,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":8,"Value":295,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4871405,"Value":176922123,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":210,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4871405,"Executor Run Time":8,"Executor CPU Time":7716743,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":170,"Index":38,"Attempt":0,"Launch Time":1547788472903,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":168,"Index":36,"Attempt":0,"Launch Time":1547788472877,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472904,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1556","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"93060956","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":26738688,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":137571,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":8759637,"Value":271837090,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":8,"Value":303,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5927866,"Value":182849989,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":217,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":5927866,"Executor Run Time":8,"Executor CPU Time":8759637,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":171,"Index":39,"Attempt":0,"Launch Time":1547788472917,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":170,"Index":38,"Attempt":0,"Launch Time":1547788472903,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472917,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"2","Value":"1558","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"93323099","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":27000832,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":141188,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5642002,"Value":277479092,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":5,"Value":308,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3540817,"Value":186390806,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":221,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3540817,"Executor Run Time":5,"Executor CPU Time":5642002,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":172,"Index":40,"Attempt":0,"Launch Time":1547788472935,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":171,"Index":39,"Attempt":0,"Launch Time":1547788472917,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472936,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1562","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"93585242","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":27262976,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":144805,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7661206,"Value":285140298,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":8,"Value":316,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5642762,"Value":192033568,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":226,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5642762,"Executor Run Time":8,"Executor CPU Time":7661206,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":173,"Index":41,"Attempt":0,"Launch Time":1547788472948,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":172,"Index":40,"Attempt":0,"Launch Time":1547788472935,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472948,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"2","Value":"1564","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"93847385","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":27525120,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":148422,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5302931,"Value":290443229,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":5,"Value":321,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3436848,"Value":195470416,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":229,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3436848,"Executor Run Time":5,"Executor CPU Time":5302931,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":174,"Index":42,"Attempt":0,"Launch Time":1547788472961,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":173,"Index":41,"Attempt":0,"Launch Time":1547788472948,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472961,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1567","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"94109528","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":27787264,"Internal":true,"Count Failed Values":true},{"ID":370,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3660,"Value":152082,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5739070,"Value":296182299,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":5,"Value":326,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3562405,"Value":199032821,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":233,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3562405,"Executor Run Time":5,"Executor CPU Time":5739070,"Result Size":3660,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":175,"Index":44,"Attempt":0,"Launch Time":1547788472976,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":174,"Index":42,"Attempt":0,"Launch Time":1547788472961,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472976,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1570","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"94371671","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":28049408,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":155699,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5963709,"Value":302146008,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":6,"Value":332,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3636041,"Value":202668862,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":236,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3636041,"Executor Run Time":6,"Executor CPU Time":5963709,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":176,"Index":45,"Attempt":0,"Launch Time":1547788472992,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":175,"Index":44,"Attempt":0,"Launch Time":1547788472976,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472993,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1573","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"94633814","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":28311552,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":159316,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":6565669,"Value":308711677,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":7,"Value":339,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5404627,"Value":208073489,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":241,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5404627,"Executor Run Time":7,"Executor CPU Time":6565669,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":177,"Index":46,"Attempt":0,"Launch Time":1547788473009,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":176,"Index":45,"Attempt":0,"Launch Time":1547788472992,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473009,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1577","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"94895957","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":28573696,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":162933,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7033349,"Value":315745026,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":6,"Value":345,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5074273,"Value":213147762,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":247,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5074273,"Executor Run Time":6,"Executor CPU Time":7033349,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":178,"Index":47,"Attempt":0,"Launch Time":1547788473026,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":177,"Index":46,"Attempt":0,"Launch Time":1547788473009,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473026,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1581","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"95158100","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":28835840,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":166550,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7390998,"Value":323136024,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":8,"Value":353,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4966515,"Value":218114277,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":251,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4966515,"Executor Run Time":8,"Executor CPU Time":7390998,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":179,"Index":48,"Attempt":0,"Launch Time":1547788473032,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":169,"Index":37,"Attempt":0,"Launch Time":1547788472878,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473032,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1586","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"95420243","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":29097984,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":170167,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7742581,"Value":330878605,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":8,"Value":361,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4684364,"Value":222798641,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":256,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4684364,"Executor Run Time":8,"Executor CPU Time":7742581,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":180,"Index":49,"Attempt":0,"Launch Time":1547788473044,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":178,"Index":47,"Attempt":0,"Launch Time":1547788473026,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473045,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1590","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"95682386","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":29360128,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":173784,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7257991,"Value":338136596,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":8,"Value":369,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5307271,"Value":228105912,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":261,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5307271,"Executor Run Time":8,"Executor CPU Time":7257991,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":181,"Index":50,"Attempt":0,"Launch Time":1547788473052,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":179,"Index":48,"Attempt":0,"Launch Time":1547788473032,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473052,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1595","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"95944529","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":29622272,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":177401,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":8473342,"Value":346609938,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":9,"Value":378,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4873441,"Value":232979353,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":266,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4873441,"Executor Run Time":9,"Executor CPU Time":8473342,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":182,"Index":51,"Attempt":0,"Launch Time":1547788473065,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":180,"Index":49,"Attempt":0,"Launch Time":1547788473044,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473065,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"1601","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"96206672","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":29884416,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":181018,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":9595563,"Value":356205501,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":10,"Value":388,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5409968,"Value":238389321,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":271,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5409968,"Executor Run Time":10,"Executor CPU Time":9595563,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":183,"Index":52,"Attempt":0,"Launch Time":1547788473077,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":181,"Index":50,"Attempt":0,"Launch Time":1547788473052,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473077,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1606","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"96468815","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":30146560,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":184635,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":8216175,"Value":364421676,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":8,"Value":396,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":12085600,"Value":250474921,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":12,"Value":283,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":12,"Executor Deserialize CPU Time":12085600,"Executor Run Time":8,"Executor CPU Time":8216175,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":184,"Index":53,"Attempt":0,"Launch Time":1547788473087,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":182,"Index":51,"Attempt":0,"Launch Time":1547788473065,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473087,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"1612","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"96730958","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":30408704,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":188252,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":9744028,"Value":374165704,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":10,"Value":406,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7027353,"Value":257502274,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":290,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":7027353,"Executor Run Time":10,"Executor CPU Time":9744028,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":185,"Index":54,"Attempt":0,"Launch Time":1547788473098,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":183,"Index":52,"Attempt":0,"Launch Time":1547788473077,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473099,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"1618","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"96993101","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":30670848,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":191869,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7420363,"Value":381586067,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":9,"Value":415,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4053871,"Value":261556145,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":294,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4053871,"Executor Run Time":9,"Executor CPU Time":7420363,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":186,"Index":55,"Attempt":0,"Launch Time":1547788473108,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":184,"Index":53,"Attempt":0,"Launch Time":1547788473087,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473109,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1623","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"97255244","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":30932992,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":195486,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":8678513,"Value":390264580,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":9,"Value":424,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6978523,"Value":268534668,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":301,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":6978523,"Executor Run Time":9,"Executor CPU Time":8678513,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":187,"Index":56,"Attempt":0,"Launch Time":1547788473117,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":185,"Index":54,"Attempt":0,"Launch Time":1547788473098,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473117,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1628","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"97517387","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":31195136,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":199103,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7960536,"Value":398225116,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":7,"Value":431,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4604373,"Value":273139041,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":307,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":4604373,"Executor Run Time":7,"Executor CPU Time":7960536,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":188,"Index":57,"Attempt":0,"Launch Time":1547788473135,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":187,"Index":56,"Attempt":0,"Launch Time":1547788473117,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473136,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1633","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"97779530","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":31457280,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":202720,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":8225881,"Value":406450997,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":8,"Value":439,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4682601,"Value":277821642,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":312,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4682601,"Executor Run Time":8,"Executor CPU Time":8225881,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":189,"Index":58,"Attempt":0,"Launch Time":1547788473137,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":186,"Index":55,"Attempt":0,"Launch Time":1547788473108,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473137,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"1639","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"98041673","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":31719424,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":206337,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":9838423,"Value":416289420,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":11,"Value":450,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7232048,"Value":285053690,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":8,"Value":320,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":8,"Executor Deserialize CPU Time":7232048,"Executor Run Time":11,"Executor CPU Time":9838423,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":190,"Index":59,"Attempt":0,"Launch Time":1547788473153,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":188,"Index":57,"Attempt":0,"Launch Time":1547788473135,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473153,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1644","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"98303816","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":31981568,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":209954,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":8000257,"Value":424289677,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":7,"Value":457,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4745183,"Value":289798873,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":325,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4745183,"Executor Run Time":7,"Executor CPU Time":8000257,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":191,"Index":60,"Attempt":0,"Launch Time":1547788473158,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":189,"Index":58,"Attempt":0,"Launch Time":1547788473137,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473158,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"1650","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"98565959","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":32243712,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":213571,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":8960938,"Value":433250615,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":8,"Value":465,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6215634,"Value":296014507,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":332,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":6215634,"Executor Run Time":8,"Executor CPU Time":8960938,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":192,"Index":61,"Attempt":0,"Launch Time":1547788473176,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":190,"Index":59,"Attempt":0,"Launch Time":1547788473153,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473177,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1655","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"98828102","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":32505856,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":217188,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":8143954,"Value":441394569,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":9,"Value":474,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4799811,"Value":300814318,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":337,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4799811,"Executor Run Time":9,"Executor CPU Time":8143954,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":193,"Index":62,"Attempt":0,"Launch Time":1547788473179,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":191,"Index":60,"Attempt":0,"Launch Time":1547788473158,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473180,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1660","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"99090245","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":32768000,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":220805,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":8037261,"Value":449431830,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":9,"Value":483,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7208632,"Value":308022950,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":344,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":7208632,"Executor Run Time":9,"Executor CPU Time":8037261,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":194,"Index":63,"Attempt":0,"Launch Time":1547788473198,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":192,"Index":61,"Attempt":0,"Launch Time":1547788473176,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473198,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"1666","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"99352388","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":33030144,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":224422,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":8361505,"Value":457793335,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":9,"Value":492,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5062107,"Value":313085057,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":351,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":5062107,"Executor Run Time":9,"Executor CPU Time":8361505,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":195,"Index":64,"Attempt":0,"Launch Time":1547788473200,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":193,"Index":62,"Attempt":0,"Launch Time":1547788473179,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473201,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"1672","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"99614531","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":33292288,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":228039,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":9530500,"Value":467323835,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":10,"Value":502,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5959460,"Value":319044517,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":356,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5959460,"Executor Run Time":10,"Executor CPU Time":9530500,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":196,"Index":65,"Attempt":0,"Launch Time":1547788473220,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":195,"Index":64,"Attempt":0,"Launch Time":1547788473200,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473220,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1676","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"99876674","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":33554432,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":231656,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7830036,"Value":475153871,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":8,"Value":510,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5342625,"Value":324387142,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":362,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5342625,"Executor Run Time":8,"Executor CPU Time":7830036,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":197,"Index":66,"Attempt":0,"Launch Time":1547788473234,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":194,"Index":63,"Attempt":0,"Launch Time":1547788473198,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473235,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"15","Value":"1691","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"100138817","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":33816576,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":235273,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":12073310,"Value":487227181,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":20,"Value":530,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5305109,"Value":329692251,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":369,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":5305109,"Executor Run Time":20,"Executor CPU Time":12073310,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":198,"Index":67,"Attempt":0,"Launch Time":1547788473239,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":196,"Index":65,"Attempt":0,"Launch Time":1547788473220,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473240,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1696","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"100400960","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":34078720,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":238890,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":8041348,"Value":495268529,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":8,"Value":538,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5597838,"Value":335290089,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":375,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5597838,"Executor Run Time":8,"Executor CPU Time":8041348,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":199,"Index":68,"Attempt":0,"Launch Time":1547788473259,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":197,"Index":66,"Attempt":0,"Launch Time":1547788473234,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473260,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"10","Value":"1706","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"100663103","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":34340864,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":242507,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":10310150,"Value":505578679,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":13,"Value":551,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5294912,"Value":340585001,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":381,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5294912,"Executor Run Time":13,"Executor CPU Time":10310150,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":200,"Index":69,"Attempt":0,"Launch Time":1547788473267,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":198,"Index":67,"Attempt":0,"Launch Time":1547788473239,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473267,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"1712","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"100925246","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":34603008,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":246124,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":11581666,"Value":517160345,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":13,"Value":564,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7177709,"Value":347762710,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":388,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":7177709,"Executor Run Time":13,"Executor CPU Time":11581666,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":201,"Index":70,"Attempt":0,"Launch Time":1547788473279,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":199,"Index":68,"Attempt":0,"Launch Time":1547788473259,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473279,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1717","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"101187389","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":34865152,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":249741,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":8196414,"Value":525356759,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":9,"Value":573,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4995663,"Value":352758373,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":394,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":4995663,"Executor Run Time":9,"Executor CPU Time":8196414,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":202,"Index":71,"Attempt":0,"Launch Time":1547788473294,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":200,"Index":69,"Attempt":0,"Launch Time":1547788473267,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473294,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"1723","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"101449532","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":35127296,"Internal":true,"Count Failed Values":true},{"ID":370,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":4,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3660,"Value":253401,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":11710046,"Value":537066805,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":12,"Value":585,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":9483809,"Value":362242182,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":9,"Value":403,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":9,"Executor Deserialize CPU Time":9483809,"Executor Run Time":12,"Executor CPU Time":11710046,"Result Size":3660,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":203,"Index":72,"Attempt":0,"Launch Time":1547788473297,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":201,"Index":70,"Attempt":0,"Launch Time":1547788473279,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473298,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1727","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"101711675","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":35389440,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":257018,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":6968674,"Value":544035479,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":7,"Value":592,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4770014,"Value":367012196,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":409,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":4770014,"Executor Run Time":7,"Executor CPU Time":6968674,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":204,"Index":73,"Attempt":0,"Launch Time":1547788473312,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":202,"Index":71,"Attempt":0,"Launch Time":1547788473294,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473312,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1730","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"101973818","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":35651584,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":260635,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":6861059,"Value":550896538,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":7,"Value":599,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7050527,"Value":374062723,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":416,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":7050527,"Executor Run Time":7,"Executor CPU Time":6861059,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":205,"Index":74,"Attempt":0,"Launch Time":1547788473317,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":203,"Index":72,"Attempt":0,"Launch Time":1547788473297,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473318,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"7","Value":"1737","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"102235961","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":35913728,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":264252,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7897659,"Value":558794197,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":11,"Value":610,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4454965,"Value":378517688,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":421,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4454965,"Executor Run Time":11,"Executor CPU Time":7897659,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":204,"Index":73,"Attempt":0,"Launch Time":1547788473312,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473326,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"2","Value":"1739","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"102498104","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":36175872,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":267869,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5088294,"Value":563882491,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":4,"Value":614,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4281314,"Value":382799002,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":426,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4281314,"Executor Run Time":4,"Executor CPU Time":5088294,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":205,"Index":74,"Attempt":0,"Launch Time":1547788473317,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473334,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1742","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"102760247","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":36438016,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":271486,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":6293560,"Value":570176051,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":6,"Value":620,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4340534,"Value":387139536,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":431,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4340534,"Executor Run Time":6,"Executor CPU Time":6293560,"Result Size":3617,"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":"SparkListenerStageCompleted","Stage Info":{"Stage ID":14,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":75,"RDD Info":[{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"50\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[13],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788472512,"Completion Time":1547788473335,"Accumulables":[{"ID":227,"Name":"aggregate time total (min, med, max)","Value":"-66","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":209,"Name":"records read","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Value":431,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Value":36438016,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Value":570176051,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":370,"Name":"internal.metrics.resultSerializationTime","Value":4,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":63,"Internal":true,"Count Failed Values":true},{"ID":223,"Name":"duration total (min, med, max)","Value":"1742","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Value":620,"Internal":true,"Count Failed Values":true},{"ID":225,"Name":"peak memory total (min, med, max)","Value":"102760247","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":207,"Name":"local bytes read total (min, med, max)","Value":"59","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":228,"Name":"avg hash probe (min, med, max)","Value":"30","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":204,"Name":"local blocks read","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":368,"Name":"internal.metrics.resultSize","Value":271486,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Value":387139536,"Internal":true,"Count Failed Values":true},{"ID":224,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"}]}} +{"Event":"SparkListenerJobEnd","Job ID":9,"Completion Time":1547788473336,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":4,"time":1547788473342} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":5,"description":"createOrReplaceTempView at SparkSQLExample.scala:122","details":"org.apache.spark.sql.Dataset.createOrReplaceTempView(Dataset.scala:3021)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:122)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\nCreateViewCommand `people`, false, true, LocalTempView\n +- Relation[age#7L,name#8] json\n\n== Analyzed Logical Plan ==\n\nCreateViewCommand `people`, false, true, LocalTempView\n +- Relation[age#7L,name#8] json\n\n== Optimized Logical Plan ==\nCreateViewCommand `people`, false, true, LocalTempView\n +- Relation[age#7L,name#8] json\n\n== Physical Plan ==\nExecute CreateViewCommand\n +- CreateViewCommand `people`, false, true, LocalTempView\n +- Relation[age#7L,name#8] json\n","sparkPlanInfo":{"nodeName":"Execute CreateViewCommand","simpleString":"Execute CreateViewCommand","children":[],"metadata":{},"metrics":[]},"time":1547788473539} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":5,"time":1547788473541} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":6,"description":"show at SparkSQLExample.scala:125","details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:125)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#7L as string) AS age#76, cast(name#8 as string) AS name#77]\n +- Project [age#7L, name#8]\n +- SubqueryAlias `people`\n +- Relation[age#7L,name#8] json\n\n== Analyzed Logical Plan ==\nage: string, name: string\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#7L as string) AS age#76, cast(name#8 as string) AS name#77]\n +- Project [age#7L, name#8]\n +- SubqueryAlias `people`\n +- Relation[age#7L,name#8] json\n\n== Optimized Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#7L as string) AS age#76, name#8]\n +- Relation[age#7L,name#8] json\n\n== Physical Plan ==\nCollectLimit 21\n+- *(1) Project [cast(age#7L as string) AS age#76, name#8]\n +- *(1) FileScan json [age#7L,name#8] Batched: false, DataFilters: [], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n","sparkPlanInfo":{"nodeName":"CollectLimit","simpleString":"CollectLimit 21","children":[{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"Project","simpleString":"Project [cast(age#7L as string) AS age#76, name#8]","children":[{"nodeName":"Scan json ","simpleString":"FileScan json [age#7L,name#8] Batched: false, DataFilters: [], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct","children":[],"metadata":{"Location":"InMemoryFileIndex[hdfs://node6:8020/user/spark/examples/src/main/resources/people.json]","ReadSchema":"struct","Format":"JSON","Batched":"false","PartitionFilters":"[]","PushedFilters":"[]","DataFilters":"[]"},"metrics":[{"name":"number of output rows","accumulatorId":400,"metricType":"sum"},{"name":"number of files","accumulatorId":401,"metricType":"sum"},{"name":"metadata time","accumulatorId":402,"metricType":"sum"},{"name":"scan time total (min, med, max)","accumulatorId":403,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration total (min, med, max)","accumulatorId":399,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":397,"metricType":"sum"},{"name":"shuffle write time total (min, med, max)","accumulatorId":398,"metricType":"nsTiming"},{"name":"records read","accumulatorId":395,"metricType":"sum"},{"name":"local bytes read total (min, med, max)","accumulatorId":393,"metricType":"size"},{"name":"fetch wait time total (min, med, max)","accumulatorId":394,"metricType":"timing"},{"name":"remote bytes read total (min, med, max)","accumulatorId":391,"metricType":"size"},{"name":"local blocks read","accumulatorId":390,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":389,"metricType":"sum"},{"name":"remote bytes read to disk total (min, med, max)","accumulatorId":392,"metricType":"size"},{"name":"shuffle bytes written total (min, med, max)","accumulatorId":396,"metricType":"size"}]},"time":1547788473656} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":6,"accumUpdates":[[401,1],[402,0]]} +{"Event":"SparkListenerJobStart","Job ID":10,"Submission Time":1547788473735,"Stage Infos":[{"Stage ID":15,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:125","Number of Tasks":1,"RDD Info":[{"RDD ID":29,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"66\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:125","Parent IDs":[28],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":28,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"65\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:125","Parent IDs":[27],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":26,"Name":"FileScanRDD","Scope":"{\"id\":\"62\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:125","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":27,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"62\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:125","Parent IDs":[26],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:125)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[15],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"6","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":15,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:125","Number of Tasks":1,"RDD Info":[{"RDD ID":29,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"66\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:125","Parent IDs":[28],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":28,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"65\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:125","Parent IDs":[27],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":26,"Name":"FileScanRDD","Scope":"{\"id\":\"62\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:125","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":27,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"62\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:125","Parent IDs":[26],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:125)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788473736,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"6","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerTaskStart","Stage ID":15,"Stage Attempt ID":0,"Task Info":{"Task ID":206,"Index":0,"Attempt":0,"Launch Time":1547788473742,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":15,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":206,"Index":0,"Attempt":0,"Launch Time":1547788473742,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473812,"Failed":false,"Killed":false,"Accumulables":[{"ID":399,"Name":"duration total (min, med, max)","Update":"45","Value":"44","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":400,"Name":"number of output rows","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":426,"Name":"internal.metrics.input.recordsRead","Update":3,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":425,"Name":"internal.metrics.input.bytesRead","Update":73,"Value":73,"Internal":true,"Count Failed Values":true},{"ID":408,"Name":"internal.metrics.resultSize","Update":1229,"Value":1229,"Internal":true,"Count Failed Values":true},{"ID":407,"Name":"internal.metrics.executorCpuTime","Update":37928680,"Value":37928680,"Internal":true,"Count Failed Values":true},{"ID":406,"Name":"internal.metrics.executorRunTime","Update":49,"Value":49,"Internal":true,"Count Failed Values":true},{"ID":405,"Name":"internal.metrics.executorDeserializeCpuTime","Update":10668425,"Value":10668425,"Internal":true,"Count Failed Values":true},{"ID":404,"Name":"internal.metrics.executorDeserializeTime","Update":15,"Value":15,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":15,"Executor Deserialize CPU Time":10668425,"Executor Run Time":49,"Executor CPU Time":37928680,"Result Size":1229,"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":73,"Records Read":3},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":15,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:125","Number of Tasks":1,"RDD Info":[{"RDD ID":29,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"66\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:125","Parent IDs":[28],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":28,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"65\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:125","Parent IDs":[27],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":26,"Name":"FileScanRDD","Scope":"{\"id\":\"62\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:125","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":27,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"62\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:125","Parent IDs":[26],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:125)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788473736,"Completion Time":1547788473813,"Accumulables":[{"ID":406,"Name":"internal.metrics.executorRunTime","Value":49,"Internal":true,"Count Failed Values":true},{"ID":400,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":405,"Name":"internal.metrics.executorDeserializeCpuTime","Value":10668425,"Internal":true,"Count Failed Values":true},{"ID":399,"Name":"duration total (min, med, max)","Value":"44","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":426,"Name":"internal.metrics.input.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":408,"Name":"internal.metrics.resultSize","Value":1229,"Internal":true,"Count Failed Values":true},{"ID":404,"Name":"internal.metrics.executorDeserializeTime","Value":15,"Internal":true,"Count Failed Values":true},{"ID":425,"Name":"internal.metrics.input.bytesRead","Value":73,"Internal":true,"Count Failed Values":true},{"ID":407,"Name":"internal.metrics.executorCpuTime","Value":37928680,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":10,"Completion Time":1547788473813,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":6,"time":1547788473815} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":7,"description":"createGlobalTempView at SparkSQLExample.scala:137","details":"org.apache.spark.sql.Dataset.createGlobalTempView(Dataset.scala:3040)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:137)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\nCreateViewCommand `people`, false, false, GlobalTempView\n +- Relation[age#7L,name#8] json\n\n== Analyzed Logical Plan ==\n\nCreateViewCommand `people`, false, false, GlobalTempView\n +- Relation[age#7L,name#8] json\n\n== Optimized Logical Plan ==\nCreateViewCommand `people`, false, false, GlobalTempView\n +- Relation[age#7L,name#8] json\n\n== Physical Plan ==\nExecute CreateViewCommand\n +- CreateViewCommand `people`, false, false, GlobalTempView\n +- Relation[age#7L,name#8] json\n","sparkPlanInfo":{"nodeName":"Execute CreateViewCommand","simpleString":"Execute CreateViewCommand","children":[],"metadata":{},"metrics":[]},"time":1547788473824} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":7,"time":1547788473824} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":8,"description":"show at SparkSQLExample.scala:140","details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:140)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#7L as string) AS age#89, cast(name#8 as string) AS name#90]\n +- Project [age#7L, name#8]\n +- SubqueryAlias `global_temp`.`people`\n +- Relation[age#7L,name#8] json\n\n== Analyzed Logical Plan ==\nage: string, name: string\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#7L as string) AS age#89, cast(name#8 as string) AS name#90]\n +- Project [age#7L, name#8]\n +- SubqueryAlias `global_temp`.`people`\n +- Relation[age#7L,name#8] json\n\n== Optimized Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#7L as string) AS age#89, name#8]\n +- Relation[age#7L,name#8] json\n\n== Physical Plan ==\nCollectLimit 21\n+- *(1) Project [cast(age#7L as string) AS age#89, name#8]\n +- *(1) FileScan json [age#7L,name#8] Batched: false, DataFilters: [], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n","sparkPlanInfo":{"nodeName":"CollectLimit","simpleString":"CollectLimit 21","children":[{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"Project","simpleString":"Project [cast(age#7L as string) AS age#89, name#8]","children":[{"nodeName":"Scan json ","simpleString":"FileScan json [age#7L,name#8] Batched: false, DataFilters: [], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct","children":[],"metadata":{"Location":"InMemoryFileIndex[hdfs://node6:8020/user/spark/examples/src/main/resources/people.json]","ReadSchema":"struct","Format":"JSON","Batched":"false","PartitionFilters":"[]","PushedFilters":"[]","DataFilters":"[]"},"metrics":[{"name":"number of output rows","accumulatorId":440,"metricType":"sum"},{"name":"number of files","accumulatorId":441,"metricType":"sum"},{"name":"metadata time","accumulatorId":442,"metricType":"sum"},{"name":"scan time total (min, med, max)","accumulatorId":443,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration total (min, med, max)","accumulatorId":439,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":437,"metricType":"sum"},{"name":"shuffle write time total (min, med, max)","accumulatorId":438,"metricType":"nsTiming"},{"name":"records read","accumulatorId":435,"metricType":"sum"},{"name":"local bytes read total (min, med, max)","accumulatorId":433,"metricType":"size"},{"name":"fetch wait time total (min, med, max)","accumulatorId":434,"metricType":"timing"},{"name":"remote bytes read total (min, med, max)","accumulatorId":431,"metricType":"size"},{"name":"local blocks read","accumulatorId":430,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":429,"metricType":"sum"},{"name":"remote bytes read to disk total (min, med, max)","accumulatorId":432,"metricType":"size"},{"name":"shuffle bytes written total (min, med, max)","accumulatorId":436,"metricType":"size"}]},"time":1547788473855} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":8,"accumUpdates":[[441,1],[442,0]]} +{"Event":"SparkListenerJobStart","Job ID":11,"Submission Time":1547788473893,"Stage Infos":[{"Stage ID":16,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:140","Number of Tasks":1,"RDD Info":[{"RDD ID":33,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"75\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:140","Parent IDs":[32],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":32,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"74\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:140","Parent IDs":[31],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":31,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"71\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:140","Parent IDs":[30],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":30,"Name":"FileScanRDD","Scope":"{\"id\":\"71\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:140","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:140)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[16],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"8","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":16,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:140","Number of Tasks":1,"RDD Info":[{"RDD ID":33,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"75\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:140","Parent IDs":[32],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":32,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"74\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:140","Parent IDs":[31],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":31,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"71\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:140","Parent IDs":[30],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":30,"Name":"FileScanRDD","Scope":"{\"id\":\"71\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:140","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:140)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788473894,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"8","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":207,"Index":0,"Attempt":0,"Launch Time":1547788473904,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":207,"Index":0,"Attempt":0,"Launch Time":1547788473904,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473959,"Failed":false,"Killed":false,"Accumulables":[{"ID":439,"Name":"duration total (min, med, max)","Update":"36","Value":"35","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":440,"Name":"number of output rows","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":466,"Name":"internal.metrics.input.recordsRead","Update":3,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":465,"Name":"internal.metrics.input.bytesRead","Update":73,"Value":73,"Internal":true,"Count Failed Values":true},{"ID":448,"Name":"internal.metrics.resultSize","Update":1229,"Value":1229,"Internal":true,"Count Failed Values":true},{"ID":447,"Name":"internal.metrics.executorCpuTime","Update":31052290,"Value":31052290,"Internal":true,"Count Failed Values":true},{"ID":446,"Name":"internal.metrics.executorRunTime","Update":38,"Value":38,"Internal":true,"Count Failed Values":true},{"ID":445,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6114378,"Value":6114378,"Internal":true,"Count Failed Values":true},{"ID":444,"Name":"internal.metrics.executorDeserializeTime","Update":12,"Value":12,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":12,"Executor Deserialize CPU Time":6114378,"Executor Run Time":38,"Executor CPU Time":31052290,"Result Size":1229,"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":73,"Records Read":3},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":16,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:140","Number of Tasks":1,"RDD Info":[{"RDD ID":33,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"75\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:140","Parent IDs":[32],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":32,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"74\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:140","Parent IDs":[31],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":31,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"71\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:140","Parent IDs":[30],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":30,"Name":"FileScanRDD","Scope":"{\"id\":\"71\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:140","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:140)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788473894,"Completion Time":1547788473960,"Accumulables":[{"ID":445,"Name":"internal.metrics.executorDeserializeCpuTime","Value":6114378,"Internal":true,"Count Failed Values":true},{"ID":439,"Name":"duration total (min, med, max)","Value":"35","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":448,"Name":"internal.metrics.resultSize","Value":1229,"Internal":true,"Count Failed Values":true},{"ID":465,"Name":"internal.metrics.input.bytesRead","Value":73,"Internal":true,"Count Failed Values":true},{"ID":447,"Name":"internal.metrics.executorCpuTime","Value":31052290,"Internal":true,"Count Failed Values":true},{"ID":444,"Name":"internal.metrics.executorDeserializeTime","Value":12,"Internal":true,"Count Failed Values":true},{"ID":446,"Name":"internal.metrics.executorRunTime","Value":38,"Internal":true,"Count Failed Values":true},{"ID":440,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":466,"Name":"internal.metrics.input.recordsRead","Value":3,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":11,"Completion Time":1547788473960,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":8,"time":1547788473961} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":9,"description":"show at SparkSQLExample.scala:150","details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:150)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#7L as string) AS age#102, cast(name#8 as string) AS name#103]\n +- Project [age#7L, name#8]\n +- SubqueryAlias `global_temp`.`people`\n +- Relation[age#7L,name#8] json\n\n== Analyzed Logical Plan ==\nage: string, name: string\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#7L as string) AS age#102, cast(name#8 as string) AS name#103]\n +- Project [age#7L, name#8]\n +- SubqueryAlias `global_temp`.`people`\n +- Relation[age#7L,name#8] json\n\n== Optimized Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#7L as string) AS age#102, name#8]\n +- Relation[age#7L,name#8] json\n\n== Physical Plan ==\nCollectLimit 21\n+- *(1) Project [cast(age#7L as string) AS age#102, name#8]\n +- *(1) FileScan json [age#7L,name#8] Batched: false, DataFilters: [], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n","sparkPlanInfo":{"nodeName":"CollectLimit","simpleString":"CollectLimit 21","children":[{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"Project","simpleString":"Project [cast(age#7L as string) AS age#102, name#8]","children":[{"nodeName":"Scan json ","simpleString":"FileScan json [age#7L,name#8] Batched: false, DataFilters: [], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct","children":[],"metadata":{"Location":"InMemoryFileIndex[hdfs://node6:8020/user/spark/examples/src/main/resources/people.json]","ReadSchema":"struct","Format":"JSON","Batched":"false","PartitionFilters":"[]","PushedFilters":"[]","DataFilters":"[]"},"metrics":[{"name":"number of output rows","accumulatorId":480,"metricType":"sum"},{"name":"number of files","accumulatorId":481,"metricType":"sum"},{"name":"metadata time","accumulatorId":482,"metricType":"sum"},{"name":"scan time total (min, med, max)","accumulatorId":483,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration total (min, med, max)","accumulatorId":479,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":477,"metricType":"sum"},{"name":"shuffle write time total (min, med, max)","accumulatorId":478,"metricType":"nsTiming"},{"name":"records read","accumulatorId":475,"metricType":"sum"},{"name":"local bytes read total (min, med, max)","accumulatorId":473,"metricType":"size"},{"name":"fetch wait time total (min, med, max)","accumulatorId":474,"metricType":"timing"},{"name":"remote bytes read total (min, med, max)","accumulatorId":471,"metricType":"size"},{"name":"local blocks read","accumulatorId":470,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":469,"metricType":"sum"},{"name":"remote bytes read to disk total (min, med, max)","accumulatorId":472,"metricType":"size"},{"name":"shuffle bytes written total (min, med, max)","accumulatorId":476,"metricType":"size"}]},"time":1547788473996} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":9,"accumUpdates":[[481,1],[482,0]]} +{"Event":"SparkListenerJobStart","Job ID":12,"Submission Time":1547788474036,"Stage Infos":[{"Stage ID":17,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:150","Number of Tasks":1,"RDD Info":[{"RDD ID":37,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"84\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:150","Parent IDs":[36],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":34,"Name":"FileScanRDD","Scope":"{\"id\":\"80\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:150","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":36,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"83\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:150","Parent IDs":[35],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":35,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"80\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:150","Parent IDs":[34],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:150)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[17],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"9","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":17,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:150","Number of Tasks":1,"RDD Info":[{"RDD ID":37,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"84\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:150","Parent IDs":[36],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":34,"Name":"FileScanRDD","Scope":"{\"id\":\"80\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:150","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":36,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"83\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:150","Parent IDs":[35],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":35,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"80\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:150","Parent IDs":[34],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:150)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788474037,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"9","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerTaskStart","Stage ID":17,"Stage Attempt ID":0,"Task Info":{"Task ID":208,"Index":0,"Attempt":0,"Launch Time":1547788474043,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":17,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":208,"Index":0,"Attempt":0,"Launch Time":1547788474043,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788474109,"Failed":false,"Killed":false,"Accumulables":[{"ID":479,"Name":"duration total (min, med, max)","Update":"47","Value":"46","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":480,"Name":"number of output rows","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":506,"Name":"internal.metrics.input.recordsRead","Update":3,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":505,"Name":"internal.metrics.input.bytesRead","Update":73,"Value":73,"Internal":true,"Count Failed Values":true},{"ID":488,"Name":"internal.metrics.resultSize","Update":1229,"Value":1229,"Internal":true,"Count Failed Values":true},{"ID":487,"Name":"internal.metrics.executorCpuTime","Update":41699265,"Value":41699265,"Internal":true,"Count Failed Values":true},{"ID":486,"Name":"internal.metrics.executorRunTime","Update":50,"Value":50,"Internal":true,"Count Failed Values":true},{"ID":485,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7235198,"Value":7235198,"Internal":true,"Count Failed Values":true},{"ID":484,"Name":"internal.metrics.executorDeserializeTime","Update":12,"Value":12,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":12,"Executor Deserialize CPU Time":7235198,"Executor Run Time":50,"Executor CPU Time":41699265,"Result Size":1229,"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":73,"Records Read":3},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":17,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:150","Number of Tasks":1,"RDD Info":[{"RDD ID":37,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"84\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:150","Parent IDs":[36],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":34,"Name":"FileScanRDD","Scope":"{\"id\":\"80\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:150","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":36,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"83\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:150","Parent IDs":[35],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":35,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"80\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:150","Parent IDs":[34],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:150)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788474037,"Completion Time":1547788474110,"Accumulables":[{"ID":505,"Name":"internal.metrics.input.bytesRead","Value":73,"Internal":true,"Count Failed Values":true},{"ID":487,"Name":"internal.metrics.executorCpuTime","Value":41699265,"Internal":true,"Count Failed Values":true},{"ID":480,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":486,"Name":"internal.metrics.executorRunTime","Value":50,"Internal":true,"Count Failed Values":true},{"ID":479,"Name":"duration total (min, med, max)","Value":"46","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":506,"Name":"internal.metrics.input.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":488,"Name":"internal.metrics.resultSize","Value":1229,"Internal":true,"Count Failed Values":true},{"ID":485,"Name":"internal.metrics.executorDeserializeCpuTime","Value":7235198,"Internal":true,"Count Failed Values":true},{"ID":484,"Name":"internal.metrics.executorDeserializeTime","Value":12,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":12,"Completion Time":1547788474111,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":9,"time":1547788474112} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":10,"description":"show at SparkSQLExample.scala:166","details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runDatasetCreationExample(SparkSQLExample.scala:166)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:50)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(name#113 as string) AS name#120, cast(age#114L as string) AS age#121]\n +- LocalRelation [name#113, age#114L]\n\n== Analyzed Logical Plan ==\nname: string, age: string\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(name#113 as string) AS name#120, cast(age#114L as string) AS age#121]\n +- LocalRelation [name#113, age#114L]\n\n== Optimized Logical Plan ==\nLocalRelation [name#120, age#121]\n\n== Physical Plan ==\nLocalTableScan [name#120, age#121]\n","sparkPlanInfo":{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [name#120, age#121]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":509,"metricType":"sum"}]},"time":1547788474280} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":10,"time":1547788474298} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":11,"description":"collect at SparkSQLExample.scala:175","details":"org.apache.spark.sql.Dataset.collect(Dataset.scala:2708)\norg.apache.spark.examples.sql.SparkSQLExample$.runDatasetCreationExample(SparkSQLExample.scala:175)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:50)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\n'SerializeFromObject [input[0, int, false] AS value#134]\n+- 'MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2489/1993254639@5aaa636f, int, [StructField(value,IntegerType,false)], obj#133: int\n +- 'DeserializeToObject unresolveddeserializer(assertnotnull(upcast(getcolumnbyordinal(0, IntegerType), IntegerType, - root class: \"scala.Int\"))), obj#132: int\n +- LocalRelation [value#128]\n\n== Analyzed Logical Plan ==\nvalue: int\nSerializeFromObject [input[0, int, false] AS value#134]\n+- MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2489/1993254639@5aaa636f, int, [StructField(value,IntegerType,false)], obj#133: int\n +- DeserializeToObject assertnotnull(cast(value#128 as int)), obj#132: int\n +- LocalRelation [value#128]\n\n== Optimized Logical Plan ==\nSerializeFromObject [input[0, int, false] AS value#134]\n+- MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2489/1993254639@5aaa636f, int, [StructField(value,IntegerType,false)], obj#133: int\n +- DeserializeToObject value#128: int, obj#132: int\n +- LocalRelation [value#128]\n\n== Physical Plan ==\n*(1) SerializeFromObject [input[0, int, false] AS value#134]\n+- *(1) MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2489/1993254639@5aaa636f, obj#133: int\n +- *(1) DeserializeToObject value#128: int, obj#132: int\n +- LocalTableScan [value#128]\n","sparkPlanInfo":{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [input[0, int, false] AS value#134]","children":[{"nodeName":"MapElements","simpleString":"MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2489/1993254639@5aaa636f, obj#133: int","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#128: int, obj#132: int","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [value#128]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":511,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration total (min, med, max)","accumulatorId":510,"metricType":"timing"}]},"time":1547788474349} +{"Event":"SparkListenerJobStart","Job ID":13,"Submission Time":1547788474403,"Stage Infos":[{"Stage ID":18,"Stage Attempt ID":0,"Stage Name":"collect at SparkSQLExample.scala:175","Number of Tasks":2,"RDD Info":[{"RDD ID":41,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"95\",\"name\":\"mapPartitionsInternal\"}","Callsite":"collect at SparkSQLExample.scala:175","Parent IDs":[40],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":38,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"94\",\"name\":\"LocalTableScan\"}","Callsite":"collect at SparkSQLExample.scala:175","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":39,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"94\",\"name\":\"LocalTableScan\"}","Callsite":"collect at SparkSQLExample.scala:175","Parent IDs":[38],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":40,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"89\",\"name\":\"WholeStageCodegen\"}","Callsite":"collect at SparkSQLExample.scala:175","Parent IDs":[39],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.collect(Dataset.scala:2708)\norg.apache.spark.examples.sql.SparkSQLExample$.runDatasetCreationExample(SparkSQLExample.scala:175)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:50)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[18],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.rdd.scope":"{\"id\":\"96\",\"name\":\"collect\"}","spark.rdd.scope.noOverride":"true","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"11","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":18,"Stage Attempt ID":0,"Stage Name":"collect at SparkSQLExample.scala:175","Number of Tasks":2,"RDD Info":[{"RDD ID":41,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"95\",\"name\":\"mapPartitionsInternal\"}","Callsite":"collect at SparkSQLExample.scala:175","Parent IDs":[40],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":38,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"94\",\"name\":\"LocalTableScan\"}","Callsite":"collect at SparkSQLExample.scala:175","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":39,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"94\",\"name\":\"LocalTableScan\"}","Callsite":"collect at SparkSQLExample.scala:175","Parent IDs":[38],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":40,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"89\",\"name\":\"WholeStageCodegen\"}","Callsite":"collect at SparkSQLExample.scala:175","Parent IDs":[39],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.collect(Dataset.scala:2708)\norg.apache.spark.examples.sql.SparkSQLExample$.runDatasetCreationExample(SparkSQLExample.scala:175)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:50)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788474405,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.rdd.scope":"{\"id\":\"96\",\"name\":\"collect\"}","spark.rdd.scope.noOverride":"true","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"11","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerTaskStart","Stage ID":18,"Stage Attempt ID":0,"Task Info":{"Task ID":209,"Index":0,"Attempt":0,"Launch Time":1547788474411,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":18,"Stage Attempt ID":0,"Task Info":{"Task ID":210,"Index":1,"Attempt":0,"Launch Time":1547788474414,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":18,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":210,"Index":1,"Attempt":0,"Launch Time":1547788474414,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788474508,"Failed":false,"Killed":false,"Accumulables":[{"ID":511,"Name":"number of output rows","Update":"2","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":516,"Name":"internal.metrics.resultSize","Update":1266,"Value":1266,"Internal":true,"Count Failed Values":true},{"ID":515,"Name":"internal.metrics.executorCpuTime","Update":27108842,"Value":27108842,"Internal":true,"Count Failed Values":true},{"ID":514,"Name":"internal.metrics.executorRunTime","Update":31,"Value":31,"Internal":true,"Count Failed Values":true},{"ID":513,"Name":"internal.metrics.executorDeserializeCpuTime","Update":53942156,"Value":53942156,"Internal":true,"Count Failed Values":true},{"ID":512,"Name":"internal.metrics.executorDeserializeTime","Update":57,"Value":57,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":57,"Executor Deserialize CPU Time":53942156,"Executor Run Time":31,"Executor CPU Time":27108842,"Result Size":1266,"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":18,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":209,"Index":0,"Attempt":0,"Launch Time":1547788474411,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788474549,"Failed":false,"Killed":false,"Accumulables":[{"ID":510,"Name":"duration total (min, med, max)","Update":"0","Value":"-2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":511,"Name":"number of output rows","Update":"1","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":518,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":516,"Name":"internal.metrics.resultSize","Update":1302,"Value":2568,"Internal":true,"Count Failed Values":true},{"ID":515,"Name":"internal.metrics.executorCpuTime","Update":36033484,"Value":63142326,"Internal":true,"Count Failed Values":true},{"ID":514,"Name":"internal.metrics.executorRunTime","Update":45,"Value":76,"Internal":true,"Count Failed Values":true},{"ID":513,"Name":"internal.metrics.executorDeserializeCpuTime","Update":51814579,"Value":105756735,"Internal":true,"Count Failed Values":true},{"ID":512,"Name":"internal.metrics.executorDeserializeTime","Update":80,"Value":137,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":80,"Executor Deserialize CPU Time":51814579,"Executor Run Time":45,"Executor CPU Time":36033484,"Result Size":1302,"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":"SparkListenerStageCompleted","Stage Info":{"Stage ID":18,"Stage Attempt ID":0,"Stage Name":"collect at SparkSQLExample.scala:175","Number of Tasks":2,"RDD Info":[{"RDD ID":41,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"95\",\"name\":\"mapPartitionsInternal\"}","Callsite":"collect at SparkSQLExample.scala:175","Parent IDs":[40],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":38,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"94\",\"name\":\"LocalTableScan\"}","Callsite":"collect at SparkSQLExample.scala:175","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":39,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"94\",\"name\":\"LocalTableScan\"}","Callsite":"collect at SparkSQLExample.scala:175","Parent IDs":[38],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":40,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"89\",\"name\":\"WholeStageCodegen\"}","Callsite":"collect at SparkSQLExample.scala:175","Parent IDs":[39],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.collect(Dataset.scala:2708)\norg.apache.spark.examples.sql.SparkSQLExample$.runDatasetCreationExample(SparkSQLExample.scala:175)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:50)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788474405,"Completion Time":1547788474550,"Accumulables":[{"ID":514,"Name":"internal.metrics.executorRunTime","Value":76,"Internal":true,"Count Failed Values":true},{"ID":516,"Name":"internal.metrics.resultSize","Value":2568,"Internal":true,"Count Failed Values":true},{"ID":510,"Name":"duration total (min, med, max)","Value":"-2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":513,"Name":"internal.metrics.executorDeserializeCpuTime","Value":105756735,"Internal":true,"Count Failed Values":true},{"ID":512,"Name":"internal.metrics.executorDeserializeTime","Value":137,"Internal":true,"Count Failed Values":true},{"ID":515,"Name":"internal.metrics.executorCpuTime","Value":63142326,"Internal":true,"Count Failed Values":true},{"ID":518,"Name":"internal.metrics.resultSerializationTime","Value":1,"Internal":true,"Count Failed Values":true},{"ID":511,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"}]}} +{"Event":"SparkListenerJobEnd","Job ID":13,"Completion Time":1547788474551,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":11,"time":1547788474555} +{"Event":"SparkListenerJobStart","Job ID":14,"Submission Time":1547788474640,"Stage Infos":[{"Stage ID":19,"Stage Attempt ID":0,"Stage Name":"json at SparkSQLExample.scala:179","Number of Tasks":1,"RDD Info":[{"RDD ID":44,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"103\",\"name\":\"mapPartitions\"}","Callsite":"json at SparkSQLExample.scala:179","Parent IDs":[43],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":42,"Name":"FileScanRDD","Scope":"{\"id\":\"101\",\"name\":\"WholeStageCodegen\"}","Callsite":"json at SparkSQLExample.scala:179","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":43,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"101\",\"name\":\"WholeStageCodegen\"}","Callsite":"json at SparkSQLExample.scala:179","Parent IDs":[42],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameReader.json(DataFrameReader.scala:331)\norg.apache.spark.examples.sql.SparkSQLExample$.runDatasetCreationExample(SparkSQLExample.scala:179)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:50)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[19],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":19,"Stage Attempt ID":0,"Stage Name":"json at SparkSQLExample.scala:179","Number of Tasks":1,"RDD Info":[{"RDD ID":44,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"103\",\"name\":\"mapPartitions\"}","Callsite":"json at SparkSQLExample.scala:179","Parent IDs":[43],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":42,"Name":"FileScanRDD","Scope":"{\"id\":\"101\",\"name\":\"WholeStageCodegen\"}","Callsite":"json at SparkSQLExample.scala:179","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":43,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"101\",\"name\":\"WholeStageCodegen\"}","Callsite":"json at SparkSQLExample.scala:179","Parent IDs":[42],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameReader.json(DataFrameReader.scala:331)\norg.apache.spark.examples.sql.SparkSQLExample$.runDatasetCreationExample(SparkSQLExample.scala:179)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:50)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788474641,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerTaskStart","Stage ID":19,"Stage Attempt ID":0,"Task Info":{"Task ID":211,"Index":0,"Attempt":0,"Launch Time":1547788474646,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":19,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":211,"Index":0,"Attempt":0,"Launch Time":1547788474646,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788474710,"Failed":false,"Killed":false,"Accumulables":[{"ID":541,"Name":"duration total (min, med, max)","Update":"41","Value":"40","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":537,"Name":"number of output rows","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":564,"Name":"internal.metrics.input.recordsRead","Update":3,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":563,"Name":"internal.metrics.input.bytesRead","Update":73,"Value":73,"Internal":true,"Count Failed Values":true},{"ID":546,"Name":"internal.metrics.resultSize","Update":1877,"Value":1877,"Internal":true,"Count Failed Values":true},{"ID":545,"Name":"internal.metrics.executorCpuTime","Update":36440214,"Value":36440214,"Internal":true,"Count Failed Values":true},{"ID":544,"Name":"internal.metrics.executorRunTime","Update":45,"Value":45,"Internal":true,"Count Failed Values":true},{"ID":543,"Name":"internal.metrics.executorDeserializeCpuTime","Update":9970752,"Value":9970752,"Internal":true,"Count Failed Values":true},{"ID":542,"Name":"internal.metrics.executorDeserializeTime","Update":14,"Value":14,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":14,"Executor Deserialize CPU Time":9970752,"Executor Run Time":45,"Executor CPU Time":36440214,"Result Size":1877,"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":73,"Records Read":3},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":19,"Stage Attempt ID":0,"Stage Name":"json at SparkSQLExample.scala:179","Number of Tasks":1,"RDD Info":[{"RDD ID":44,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"103\",\"name\":\"mapPartitions\"}","Callsite":"json at SparkSQLExample.scala:179","Parent IDs":[43],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":42,"Name":"FileScanRDD","Scope":"{\"id\":\"101\",\"name\":\"WholeStageCodegen\"}","Callsite":"json at SparkSQLExample.scala:179","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":43,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"101\",\"name\":\"WholeStageCodegen\"}","Callsite":"json at SparkSQLExample.scala:179","Parent IDs":[42],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameReader.json(DataFrameReader.scala:331)\norg.apache.spark.examples.sql.SparkSQLExample$.runDatasetCreationExample(SparkSQLExample.scala:179)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:50)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788474641,"Completion Time":1547788474711,"Accumulables":[{"ID":541,"Name":"duration total (min, med, max)","Value":"40","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":543,"Name":"internal.metrics.executorDeserializeCpuTime","Value":9970752,"Internal":true,"Count Failed Values":true},{"ID":537,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":564,"Name":"internal.metrics.input.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":546,"Name":"internal.metrics.resultSize","Value":1877,"Internal":true,"Count Failed Values":true},{"ID":563,"Name":"internal.metrics.input.bytesRead","Value":73,"Internal":true,"Count Failed Values":true},{"ID":542,"Name":"internal.metrics.executorDeserializeTime","Value":14,"Internal":true,"Count Failed Values":true},{"ID":545,"Name":"internal.metrics.executorCpuTime","Value":36440214,"Internal":true,"Count Failed Values":true},{"ID":544,"Name":"internal.metrics.executorRunTime","Value":45,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":14,"Completion Time":1547788474711,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":12,"description":"show at SparkSQLExample.scala:180","details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runDatasetCreationExample(SparkSQLExample.scala:180)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:50)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#144L as string) AS age#155, cast(name#145 as string) AS name#156]\n +- Relation[age#144L,name#145] json\n\n== Analyzed Logical Plan ==\nage: string, name: string\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#144L as string) AS age#155, cast(name#145 as string) AS name#156]\n +- Relation[age#144L,name#145] json\n\n== Optimized Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#144L as string) AS age#155, name#145]\n +- Relation[age#144L,name#145] json\n\n== Physical Plan ==\nCollectLimit 21\n+- *(1) Project [cast(age#144L as string) AS age#155, name#145]\n +- *(1) FileScan json [age#144L,name#145] Batched: false, DataFilters: [], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n","sparkPlanInfo":{"nodeName":"CollectLimit","simpleString":"CollectLimit 21","children":[{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"Project","simpleString":"Project [cast(age#144L as string) AS age#155, name#145]","children":[{"nodeName":"Scan json ","simpleString":"FileScan json [age#144L,name#145] Batched: false, DataFilters: [], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct","children":[],"metadata":{"Location":"InMemoryFileIndex[hdfs://node6:8020/user/spark/examples/src/main/resources/people.json]","ReadSchema":"struct","Format":"JSON","Batched":"false","PartitionFilters":"[]","PushedFilters":"[]","DataFilters":"[]"},"metrics":[{"name":"number of output rows","accumulatorId":578,"metricType":"sum"},{"name":"number of files","accumulatorId":579,"metricType":"sum"},{"name":"metadata time","accumulatorId":580,"metricType":"sum"},{"name":"scan time total (min, med, max)","accumulatorId":581,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration total (min, med, max)","accumulatorId":577,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":575,"metricType":"sum"},{"name":"shuffle write time total (min, med, max)","accumulatorId":576,"metricType":"nsTiming"},{"name":"records read","accumulatorId":573,"metricType":"sum"},{"name":"local bytes read total (min, med, max)","accumulatorId":571,"metricType":"size"},{"name":"fetch wait time total (min, med, max)","accumulatorId":572,"metricType":"timing"},{"name":"remote bytes read total (min, med, max)","accumulatorId":569,"metricType":"size"},{"name":"local blocks read","accumulatorId":568,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":567,"metricType":"sum"},{"name":"remote bytes read to disk total (min, med, max)","accumulatorId":570,"metricType":"size"},{"name":"shuffle bytes written total (min, med, max)","accumulatorId":574,"metricType":"size"}]},"time":1547788474761} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":12,"accumUpdates":[[579,1],[580,0]]} +{"Event":"SparkListenerJobStart","Job ID":15,"Submission Time":1547788474797,"Stage Infos":[{"Stage ID":20,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:180","Number of Tasks":1,"RDD Info":[{"RDD ID":48,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"111\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:180","Parent IDs":[47],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":45,"Name":"FileScanRDD","Scope":"{\"id\":\"107\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:180","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":46,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"107\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:180","Parent IDs":[45],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":47,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"110\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:180","Parent IDs":[46],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runDatasetCreationExample(SparkSQLExample.scala:180)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:50)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[20],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"12","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":20,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:180","Number of Tasks":1,"RDD Info":[{"RDD ID":48,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"111\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:180","Parent IDs":[47],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":45,"Name":"FileScanRDD","Scope":"{\"id\":\"107\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:180","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":46,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"107\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:180","Parent IDs":[45],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":47,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"110\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:180","Parent IDs":[46],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runDatasetCreationExample(SparkSQLExample.scala:180)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:50)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788474798,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"12","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":212,"Index":0,"Attempt":0,"Launch Time":1547788474804,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":212,"Index":0,"Attempt":0,"Launch Time":1547788474804,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788474869,"Failed":false,"Killed":false,"Accumulables":[{"ID":577,"Name":"duration total (min, med, max)","Update":"44","Value":"43","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":578,"Name":"number of output rows","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":604,"Name":"internal.metrics.input.recordsRead","Update":3,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":603,"Name":"internal.metrics.input.bytesRead","Update":73,"Value":73,"Internal":true,"Count Failed Values":true},{"ID":586,"Name":"internal.metrics.resultSize","Update":1229,"Value":1229,"Internal":true,"Count Failed Values":true},{"ID":585,"Name":"internal.metrics.executorCpuTime","Update":39247717,"Value":39247717,"Internal":true,"Count Failed Values":true},{"ID":584,"Name":"internal.metrics.executorRunTime","Update":46,"Value":46,"Internal":true,"Count Failed Values":true},{"ID":583,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8752334,"Value":8752334,"Internal":true,"Count Failed Values":true},{"ID":582,"Name":"internal.metrics.executorDeserializeTime","Update":13,"Value":13,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":13,"Executor Deserialize CPU Time":8752334,"Executor Run Time":46,"Executor CPU Time":39247717,"Result Size":1229,"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":73,"Records Read":3},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":20,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:180","Number of Tasks":1,"RDD Info":[{"RDD ID":48,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"111\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:180","Parent IDs":[47],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":45,"Name":"FileScanRDD","Scope":"{\"id\":\"107\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:180","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":46,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"107\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:180","Parent IDs":[45],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":47,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"110\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:180","Parent IDs":[46],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runDatasetCreationExample(SparkSQLExample.scala:180)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:50)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788474798,"Completion Time":1547788474870,"Accumulables":[{"ID":577,"Name":"duration total (min, med, max)","Value":"43","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":603,"Name":"internal.metrics.input.bytesRead","Value":73,"Internal":true,"Count Failed Values":true},{"ID":585,"Name":"internal.metrics.executorCpuTime","Value":39247717,"Internal":true,"Count Failed Values":true},{"ID":582,"Name":"internal.metrics.executorDeserializeTime","Value":13,"Internal":true,"Count Failed Values":true},{"ID":584,"Name":"internal.metrics.executorRunTime","Value":46,"Internal":true,"Count Failed Values":true},{"ID":578,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":604,"Name":"internal.metrics.input.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":586,"Name":"internal.metrics.resultSize","Value":1229,"Internal":true,"Count Failed Values":true},{"ID":583,"Name":"internal.metrics.executorDeserializeCpuTime","Value":8752334,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":15,"Completion Time":1547788474870,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":12,"time":1547788474871} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":13,"description":"createOrReplaceTempView at SparkSQLExample.scala:203","details":"org.apache.spark.sql.Dataset.createOrReplaceTempView(Dataset.scala:3021)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:203)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\nCreateViewCommand `people`, false, true, LocalTempView\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).name, true, false) AS name#167, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).age AS age#168L]\n +- ExternalRDD [obj#166]\n\n== Analyzed Logical Plan ==\n\nCreateViewCommand `people`, false, true, LocalTempView\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).name, true, false) AS name#167, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).age AS age#168L]\n +- ExternalRDD [obj#166]\n\n== Optimized Logical Plan ==\nCreateViewCommand `people`, false, true, LocalTempView\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).name, true, false) AS name#167, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).age AS age#168L]\n +- ExternalRDD [obj#166]\n\n== Physical Plan ==\nExecute CreateViewCommand\n +- CreateViewCommand `people`, false, true, LocalTempView\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).name, true, false) AS name#167, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).age AS age#168L]\n +- ExternalRDD [obj#166]\n","sparkPlanInfo":{"nodeName":"Execute CreateViewCommand","simpleString":"Execute CreateViewCommand","children":[],"metadata":{},"metrics":[]},"time":1547788474960} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":13,"time":1547788474960} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":14,"description":"show at SparkSQLExample.scala:209","details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:209)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(value#179 as string) AS value#183]\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#179]\n +- MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2518/869554567@d33a96c, interface org.apache.spark.sql.Row, [StructField(name,StringType,true), StructField(age,LongType,false)], obj#178: java.lang.String\n +- DeserializeToObject createexternalrow(name#167.toString, age#168L, StructField(name,StringType,true), StructField(age,LongType,false)), obj#177: org.apache.spark.sql.Row\n +- Project [name#167, age#168L]\n +- Filter ((age#168L >= cast(13 as bigint)) && (age#168L <= cast(19 as bigint)))\n +- SubqueryAlias `people`\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).name, true, false) AS name#167, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).age AS age#168L]\n +- ExternalRDD [obj#166]\n\n== Analyzed Logical Plan ==\nvalue: string\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(value#179 as string) AS value#183]\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#179]\n +- MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2518/869554567@d33a96c, interface org.apache.spark.sql.Row, [StructField(name,StringType,true), StructField(age,LongType,false)], obj#178: java.lang.String\n +- DeserializeToObject createexternalrow(name#167.toString, age#168L, StructField(name,StringType,true), StructField(age,LongType,false)), obj#177: org.apache.spark.sql.Row\n +- Project [name#167, age#168L]\n +- Filter ((age#168L >= cast(13 as bigint)) && (age#168L <= cast(19 as bigint)))\n +- SubqueryAlias `people`\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).name, true, false) AS name#167, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).age AS age#168L]\n +- ExternalRDD [obj#166]\n\n== Optimized Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#179]\n +- MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2518/869554567@d33a96c, interface org.apache.spark.sql.Row, [StructField(name,StringType,true), StructField(age,LongType,false)], obj#178: java.lang.String\n +- DeserializeToObject createexternalrow(name#167.toString, age#168L, StructField(name,StringType,true), StructField(age,LongType,false)), obj#177: org.apache.spark.sql.Row\n +- Filter ((age#168L >= 13) && (age#168L <= 19))\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).name, true, false) AS name#167, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).age AS age#168L]\n +- ExternalRDD [obj#166]\n\n== Physical Plan ==\nCollectLimit 21\n+- *(1) SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#179]\n +- *(1) MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2518/869554567@d33a96c, obj#178: java.lang.String\n +- *(1) DeserializeToObject createexternalrow(name#167.toString, age#168L, StructField(name,StringType,true), StructField(age,LongType,false)), obj#177: org.apache.spark.sql.Row\n +- *(1) Filter ((age#168L >= 13) && (age#168L <= 19))\n +- *(1) SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).name, true, false) AS name#167, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).age AS age#168L]\n +- Scan[obj#166]\n","sparkPlanInfo":{"nodeName":"CollectLimit","simpleString":"CollectLimit 21","children":[{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#179]","children":[{"nodeName":"MapElements","simpleString":"MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2518/869554567@d33a96c, obj#178: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject createexternalrow(name#167.toString, age#168L, StructField(name,StringType,true), StructField(age,LongType,false)), obj#177: org.apache.spark.sql.Row","children":[{"nodeName":"Filter","simpleString":"Filter ((age#168L >= 13) && (age#168L <= 19))","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).name, true, false) AS name#167, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).age AS age#168L]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Scan","simpleString":"Scan[obj#166]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":619,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":618,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration total (min, med, max)","accumulatorId":617,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":615,"metricType":"sum"},{"name":"shuffle write time total (min, med, max)","accumulatorId":616,"metricType":"nsTiming"},{"name":"records read","accumulatorId":613,"metricType":"sum"},{"name":"local bytes read total (min, med, max)","accumulatorId":611,"metricType":"size"},{"name":"fetch wait time total (min, med, max)","accumulatorId":612,"metricType":"timing"},{"name":"remote bytes read total (min, med, max)","accumulatorId":609,"metricType":"size"},{"name":"local blocks read","accumulatorId":608,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":607,"metricType":"sum"},{"name":"remote bytes read to disk total (min, med, max)","accumulatorId":610,"metricType":"size"},{"name":"shuffle bytes written total (min, med, max)","accumulatorId":614,"metricType":"size"}]},"time":1547788475059} +{"Event":"SparkListenerJobStart","Job ID":16,"Submission Time":1547788475137,"Stage Infos":[{"Stage ID":21,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:209","Number of Tasks":1,"RDD Info":[{"RDD ID":56,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"128\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[55],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"118\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:200","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":53,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"126\",\"name\":\"Scan\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":55,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"127\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[54],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":54,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"119\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[53],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"117\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:199","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:209)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[21],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"14","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":21,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:209","Number of Tasks":1,"RDD Info":[{"RDD ID":56,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"128\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[55],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"118\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:200","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":53,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"126\",\"name\":\"Scan\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":55,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"127\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[54],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":54,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"119\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[53],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"117\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:199","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:209)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788475140,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"14","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerTaskStart","Stage ID":21,"Stage Attempt ID":0,"Task Info":{"Task ID":213,"Index":0,"Attempt":0,"Launch Time":1547788475183,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":21,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":213,"Index":0,"Attempt":0,"Launch Time":1547788475183,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788475422,"Failed":false,"Killed":false,"Accumulables":[{"ID":617,"Name":"duration total (min, med, max)","Update":"3","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":619,"Name":"number of output rows","Update":"2","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":642,"Name":"internal.metrics.input.recordsRead","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":641,"Name":"internal.metrics.input.bytesRead","Update":32,"Value":32,"Internal":true,"Count Failed Values":true},{"ID":624,"Name":"internal.metrics.resultSize","Update":1208,"Value":1208,"Internal":true,"Count Failed Values":true},{"ID":623,"Name":"internal.metrics.executorCpuTime","Update":106404621,"Value":106404621,"Internal":true,"Count Failed Values":true},{"ID":622,"Name":"internal.metrics.executorRunTime","Update":123,"Value":123,"Internal":true,"Count Failed Values":true},{"ID":621,"Name":"internal.metrics.executorDeserializeCpuTime","Update":46663283,"Value":46663283,"Internal":true,"Count Failed Values":true},{"ID":620,"Name":"internal.metrics.executorDeserializeTime","Update":100,"Value":100,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":100,"Executor Deserialize CPU Time":46663283,"Executor Run Time":123,"Executor CPU Time":106404621,"Result Size":1208,"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":32,"Records Read":2},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":21,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:209","Number of Tasks":1,"RDD Info":[{"RDD ID":56,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"128\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[55],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"118\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:200","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":53,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"126\",\"name\":\"Scan\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":55,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"127\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[54],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":54,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"119\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[53],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"117\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:199","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:209)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788475140,"Completion Time":1547788475423,"Accumulables":[{"ID":621,"Name":"internal.metrics.executorDeserializeCpuTime","Value":46663283,"Internal":true,"Count Failed Values":true},{"ID":642,"Name":"internal.metrics.input.recordsRead","Value":2,"Internal":true,"Count Failed Values":true},{"ID":624,"Name":"internal.metrics.resultSize","Value":1208,"Internal":true,"Count Failed Values":true},{"ID":641,"Name":"internal.metrics.input.bytesRead","Value":32,"Internal":true,"Count Failed Values":true},{"ID":623,"Name":"internal.metrics.executorCpuTime","Value":106404621,"Internal":true,"Count Failed Values":true},{"ID":617,"Name":"duration total (min, med, max)","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":620,"Name":"internal.metrics.executorDeserializeTime","Value":100,"Internal":true,"Count Failed Values":true},{"ID":622,"Name":"internal.metrics.executorRunTime","Value":123,"Internal":true,"Count Failed Values":true},{"ID":619,"Name":"number of output rows","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"}]}} +{"Event":"SparkListenerJobEnd","Job ID":16,"Completion Time":1547788475423,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":17,"Submission Time":1547788475432,"Stage Infos":[{"Stage ID":22,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:209","Number of Tasks":1,"RDD Info":[{"RDD ID":56,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"128\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[55],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"118\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:200","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":53,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"126\",\"name\":\"Scan\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":55,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"127\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[54],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":54,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"119\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[53],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"117\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:199","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:209)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[22],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"14","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":22,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:209","Number of Tasks":1,"RDD Info":[{"RDD ID":56,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"128\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[55],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"118\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:200","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":53,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"126\",\"name\":\"Scan\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":55,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"127\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[54],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":54,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"119\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[53],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"117\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:199","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:209)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788475433,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"14","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerTaskStart","Stage ID":22,"Stage Attempt ID":0,"Task Info":{"Task ID":214,"Index":0,"Attempt":0,"Launch Time":1547788475440,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":22,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":214,"Index":0,"Attempt":0,"Launch Time":1547788475440,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788475466,"Failed":false,"Killed":false,"Accumulables":[{"ID":618,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":619,"Name":"number of output rows","Update":"1","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":667,"Name":"internal.metrics.input.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":666,"Name":"internal.metrics.input.bytesRead","Update":16,"Value":16,"Internal":true,"Count Failed Values":true},{"ID":649,"Name":"internal.metrics.resultSize","Update":1242,"Value":1242,"Internal":true,"Count Failed Values":true},{"ID":648,"Name":"internal.metrics.executorCpuTime","Update":7188516,"Value":7188516,"Internal":true,"Count Failed Values":true},{"ID":647,"Name":"internal.metrics.executorRunTime","Update":10,"Value":10,"Internal":true,"Count Failed Values":true},{"ID":646,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7299108,"Value":7299108,"Internal":true,"Count Failed Values":true},{"ID":645,"Name":"internal.metrics.executorDeserializeTime","Update":11,"Value":11,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":11,"Executor Deserialize CPU Time":7299108,"Executor Run Time":10,"Executor CPU Time":7188516,"Result Size":1242,"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":16,"Records Read":1},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":22,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:209","Number of Tasks":1,"RDD Info":[{"RDD ID":56,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"128\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[55],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"118\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:200","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":53,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"126\",\"name\":\"Scan\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":55,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"127\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[54],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":54,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"119\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[53],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"117\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:199","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:209)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788475433,"Completion Time":1547788475467,"Accumulables":[{"ID":618,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":645,"Name":"internal.metrics.executorDeserializeTime","Value":11,"Internal":true,"Count Failed Values":true},{"ID":666,"Name":"internal.metrics.input.bytesRead","Value":16,"Internal":true,"Count Failed Values":true},{"ID":648,"Name":"internal.metrics.executorCpuTime","Value":7188516,"Internal":true,"Count Failed Values":true},{"ID":647,"Name":"internal.metrics.executorRunTime","Value":10,"Internal":true,"Count Failed Values":true},{"ID":646,"Name":"internal.metrics.executorDeserializeCpuTime","Value":7299108,"Internal":true,"Count Failed Values":true},{"ID":667,"Name":"internal.metrics.input.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":649,"Name":"internal.metrics.resultSize","Value":1242,"Internal":true,"Count Failed Values":true},{"ID":619,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"}]}} +{"Event":"SparkListenerJobEnd","Job ID":17,"Completion Time":1547788475467,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":14,"time":1547788475468} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":15,"description":"show at SparkSQLExample.scala:217","details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:217)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(value#190 as string) AS value#194]\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#190]\n +- MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2541/1946069198@a0fc978, interface org.apache.spark.sql.Row, [StructField(name,StringType,true), StructField(age,LongType,false)], obj#189: java.lang.String\n +- DeserializeToObject createexternalrow(name#167.toString, age#168L, StructField(name,StringType,true), StructField(age,LongType,false)), obj#188: org.apache.spark.sql.Row\n +- Project [name#167, age#168L]\n +- Filter ((age#168L >= cast(13 as bigint)) && (age#168L <= cast(19 as bigint)))\n +- SubqueryAlias `people`\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).name, true, false) AS name#167, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).age AS age#168L]\n +- ExternalRDD [obj#166]\n\n== Analyzed Logical Plan ==\nvalue: string\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(value#190 as string) AS value#194]\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#190]\n +- MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2541/1946069198@a0fc978, interface org.apache.spark.sql.Row, [StructField(name,StringType,true), StructField(age,LongType,false)], obj#189: java.lang.String\n +- DeserializeToObject createexternalrow(name#167.toString, age#168L, StructField(name,StringType,true), StructField(age,LongType,false)), obj#188: org.apache.spark.sql.Row\n +- Project [name#167, age#168L]\n +- Filter ((age#168L >= cast(13 as bigint)) && (age#168L <= cast(19 as bigint)))\n +- SubqueryAlias `people`\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).name, true, false) AS name#167, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).age AS age#168L]\n +- ExternalRDD [obj#166]\n\n== Optimized Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#190]\n +- MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2541/1946069198@a0fc978, interface org.apache.spark.sql.Row, [StructField(name,StringType,true), StructField(age,LongType,false)], obj#189: java.lang.String\n +- DeserializeToObject createexternalrow(name#167.toString, age#168L, StructField(name,StringType,true), StructField(age,LongType,false)), obj#188: org.apache.spark.sql.Row\n +- Filter ((age#168L >= 13) && (age#168L <= 19))\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).name, true, false) AS name#167, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).age AS age#168L]\n +- ExternalRDD [obj#166]\n\n== Physical Plan ==\nCollectLimit 21\n+- *(1) SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#190]\n +- *(1) MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2541/1946069198@a0fc978, obj#189: java.lang.String\n +- *(1) DeserializeToObject createexternalrow(name#167.toString, age#168L, StructField(name,StringType,true), StructField(age,LongType,false)), obj#188: org.apache.spark.sql.Row\n +- *(1) Filter ((age#168L >= 13) && (age#168L <= 19))\n +- *(1) SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).name, true, false) AS name#167, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).age AS age#168L]\n +- Scan[obj#166]\n","sparkPlanInfo":{"nodeName":"CollectLimit","simpleString":"CollectLimit 21","children":[{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#190]","children":[{"nodeName":"MapElements","simpleString":"MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2541/1946069198@a0fc978, obj#189: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject createexternalrow(name#167.toString, age#168L, StructField(name,StringType,true), StructField(age,LongType,false)), obj#188: org.apache.spark.sql.Row","children":[{"nodeName":"Filter","simpleString":"Filter ((age#168L >= 13) && (age#168L <= 19))","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).name, true, false) AS name#167, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).age AS age#168L]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Scan","simpleString":"Scan[obj#166]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":682,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":681,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration total (min, med, max)","accumulatorId":680,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":678,"metricType":"sum"},{"name":"shuffle write time total (min, med, max)","accumulatorId":679,"metricType":"nsTiming"},{"name":"records read","accumulatorId":676,"metricType":"sum"},{"name":"local bytes read total (min, med, max)","accumulatorId":674,"metricType":"size"},{"name":"fetch wait time total (min, med, max)","accumulatorId":675,"metricType":"timing"},{"name":"remote bytes read total (min, med, max)","accumulatorId":672,"metricType":"size"},{"name":"local blocks read","accumulatorId":671,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":670,"metricType":"sum"},{"name":"remote bytes read to disk total (min, med, max)","accumulatorId":673,"metricType":"size"},{"name":"shuffle bytes written total (min, med, max)","accumulatorId":677,"metricType":"size"}]},"time":1547788475520} +{"Event":"SparkListenerJobStart","Job ID":18,"Submission Time":1547788475541,"Stage Infos":[{"Stage ID":23,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:217","Number of Tasks":1,"RDD Info":[{"RDD ID":60,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"147\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[59],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"118\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:200","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":59,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"146\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[58],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":57,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"145\",\"name\":\"Scan\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":58,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"138\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[57],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"117\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:199","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:217)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[23],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"15","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":23,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:217","Number of Tasks":1,"RDD Info":[{"RDD ID":60,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"147\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[59],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"118\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:200","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":59,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"146\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[58],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":57,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"145\",\"name\":\"Scan\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":58,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"138\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[57],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"117\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:199","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:217)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788475542,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"15","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerTaskStart","Stage ID":23,"Stage Attempt ID":0,"Task Info":{"Task ID":215,"Index":0,"Attempt":0,"Launch Time":1547788475549,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":23,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":215,"Index":0,"Attempt":0,"Launch Time":1547788475549,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788475580,"Failed":false,"Killed":false,"Accumulables":[{"ID":680,"Name":"duration total (min, med, max)","Update":"2","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":682,"Name":"number of output rows","Update":"2","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":705,"Name":"internal.metrics.input.recordsRead","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":704,"Name":"internal.metrics.input.bytesRead","Update":32,"Value":32,"Internal":true,"Count Failed Values":true},{"ID":687,"Name":"internal.metrics.resultSize","Update":1208,"Value":1208,"Internal":true,"Count Failed Values":true},{"ID":686,"Name":"internal.metrics.executorCpuTime","Update":7352607,"Value":7352607,"Internal":true,"Count Failed Values":true},{"ID":685,"Name":"internal.metrics.executorRunTime","Update":9,"Value":9,"Internal":true,"Count Failed Values":true},{"ID":684,"Name":"internal.metrics.executorDeserializeCpuTime","Update":9475942,"Value":9475942,"Internal":true,"Count Failed Values":true},{"ID":683,"Name":"internal.metrics.executorDeserializeTime","Update":16,"Value":16,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":16,"Executor Deserialize CPU Time":9475942,"Executor Run Time":9,"Executor CPU Time":7352607,"Result Size":1208,"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":32,"Records Read":2},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":23,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:217","Number of Tasks":1,"RDD Info":[{"RDD ID":60,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"147\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[59],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"118\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:200","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":59,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"146\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[58],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":57,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"145\",\"name\":\"Scan\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":58,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"138\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[57],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"117\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:199","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:217)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788475542,"Completion Time":1547788475580,"Accumulables":[{"ID":705,"Name":"internal.metrics.input.recordsRead","Value":2,"Internal":true,"Count Failed Values":true},{"ID":687,"Name":"internal.metrics.resultSize","Value":1208,"Internal":true,"Count Failed Values":true},{"ID":684,"Name":"internal.metrics.executorDeserializeCpuTime","Value":9475942,"Internal":true,"Count Failed Values":true},{"ID":683,"Name":"internal.metrics.executorDeserializeTime","Value":16,"Internal":true,"Count Failed Values":true},{"ID":704,"Name":"internal.metrics.input.bytesRead","Value":32,"Internal":true,"Count Failed Values":true},{"ID":686,"Name":"internal.metrics.executorCpuTime","Value":7352607,"Internal":true,"Count Failed Values":true},{"ID":680,"Name":"duration total (min, med, max)","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":682,"Name":"number of output rows","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":685,"Name":"internal.metrics.executorRunTime","Value":9,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":18,"Completion Time":1547788475581,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":19,"Submission Time":1547788475593,"Stage Infos":[{"Stage ID":24,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:217","Number of Tasks":1,"RDD Info":[{"RDD ID":60,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"147\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[59],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"118\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:200","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":59,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"146\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[58],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":57,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"145\",\"name\":\"Scan\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":58,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"138\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[57],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"117\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:199","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:217)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[24],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"15","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":24,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:217","Number of Tasks":1,"RDD Info":[{"RDD ID":60,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"147\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[59],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"118\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:200","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":59,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"146\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[58],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":57,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"145\",\"name\":\"Scan\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":58,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"138\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[57],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"117\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:199","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:217)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788475594,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"15","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerTaskStart","Stage ID":24,"Stage Attempt ID":0,"Task Info":{"Task ID":216,"Index":0,"Attempt":0,"Launch Time":1547788475600,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":24,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":216,"Index":0,"Attempt":0,"Launch Time":1547788475600,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788475629,"Failed":false,"Killed":false,"Accumulables":[{"ID":681,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":682,"Name":"number of output rows","Update":"1","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":730,"Name":"internal.metrics.input.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":729,"Name":"internal.metrics.input.bytesRead","Update":16,"Value":16,"Internal":true,"Count Failed Values":true},{"ID":712,"Name":"internal.metrics.resultSize","Update":1242,"Value":1242,"Internal":true,"Count Failed Values":true},{"ID":711,"Name":"internal.metrics.executorCpuTime","Update":6828175,"Value":6828175,"Internal":true,"Count Failed Values":true},{"ID":710,"Name":"internal.metrics.executorRunTime","Update":9,"Value":9,"Internal":true,"Count Failed Values":true},{"ID":709,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8623587,"Value":8623587,"Internal":true,"Count Failed Values":true},{"ID":708,"Name":"internal.metrics.executorDeserializeTime","Update":14,"Value":14,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":14,"Executor Deserialize CPU Time":8623587,"Executor Run Time":9,"Executor CPU Time":6828175,"Result Size":1242,"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":16,"Records Read":1},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":24,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:217","Number of Tasks":1,"RDD Info":[{"RDD ID":60,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"147\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[59],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"118\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:200","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":59,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"146\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[58],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":57,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"145\",\"name\":\"Scan\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":58,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"138\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[57],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"117\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:199","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:217)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788475594,"Completion Time":1547788475630,"Accumulables":[{"ID":729,"Name":"internal.metrics.input.bytesRead","Value":16,"Internal":true,"Count Failed Values":true},{"ID":681,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":708,"Name":"internal.metrics.executorDeserializeTime","Value":14,"Internal":true,"Count Failed Values":true},{"ID":711,"Name":"internal.metrics.executorCpuTime","Value":6828175,"Internal":true,"Count Failed Values":true},{"ID":710,"Name":"internal.metrics.executorRunTime","Value":9,"Internal":true,"Count Failed Values":true},{"ID":682,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":709,"Name":"internal.metrics.executorDeserializeCpuTime","Value":8623587,"Internal":true,"Count Failed Values":true},{"ID":730,"Name":"internal.metrics.input.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":712,"Name":"internal.metrics.resultSize","Value":1242,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":19,"Completion Time":1547788475630,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":15,"time":1547788475631} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":16,"description":"collect at SparkSQLExample.scala:230","details":"org.apache.spark.sql.Dataset.collect(Dataset.scala:2708)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:230)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\n'SerializeFromObject [encodeusingserializer(input[0, java.lang.Object, true], true) AS value#201]\n+- 'MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2542/1591023561@7dd320, interface org.apache.spark.sql.Row, [StructField(name,StringType,true), StructField(age,LongType,false)], obj#200: scala.collection.immutable.Map\n +- 'DeserializeToObject unresolveddeserializer(createexternalrow(getcolumnbyordinal(0, StructField(name,StringType,true), StructField(age,LongType,false)).toString, getcolumnbyordinal(1, StructField(name,StringType,true), StructField(age,LongType,false)), StructField(name,StringType,true), StructField(age,LongType,false))), obj#199: org.apache.spark.sql.Row\n +- Project [name#167, age#168L]\n +- Filter ((age#168L >= cast(13 as bigint)) && (age#168L <= cast(19 as bigint)))\n +- SubqueryAlias `people`\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).name, true, false) AS name#167, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).age AS age#168L]\n +- ExternalRDD [obj#166]\n\n== Analyzed Logical Plan ==\nvalue: binary\nSerializeFromObject [encodeusingserializer(input[0, java.lang.Object, true], true) AS value#201]\n+- MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2542/1591023561@7dd320, interface org.apache.spark.sql.Row, [StructField(name,StringType,true), StructField(age,LongType,false)], obj#200: scala.collection.immutable.Map\n +- DeserializeToObject createexternalrow(name#167.toString, age#168L, StructField(name,StringType,true), StructField(age,LongType,false)), obj#199: org.apache.spark.sql.Row\n +- Project [name#167, age#168L]\n +- Filter ((age#168L >= cast(13 as bigint)) && (age#168L <= cast(19 as bigint)))\n +- SubqueryAlias `people`\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).name, true, false) AS name#167, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).age AS age#168L]\n +- ExternalRDD [obj#166]\n\n== Optimized Logical Plan ==\nSerializeFromObject [encodeusingserializer(input[0, java.lang.Object, true], true) AS value#201]\n+- MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2542/1591023561@7dd320, interface org.apache.spark.sql.Row, [StructField(name,StringType,true), StructField(age,LongType,false)], obj#200: scala.collection.immutable.Map\n +- DeserializeToObject createexternalrow(name#167.toString, age#168L, StructField(name,StringType,true), StructField(age,LongType,false)), obj#199: org.apache.spark.sql.Row\n +- Filter ((age#168L >= 13) && (age#168L <= 19))\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).name, true, false) AS name#167, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).age AS age#168L]\n +- ExternalRDD [obj#166]\n\n== Physical Plan ==\n*(1) SerializeFromObject [encodeusingserializer(input[0, java.lang.Object, true], true) AS value#201]\n+- *(1) MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2542/1591023561@7dd320, obj#200: scala.collection.immutable.Map\n +- *(1) DeserializeToObject createexternalrow(name#167.toString, age#168L, StructField(name,StringType,true), StructField(age,LongType,false)), obj#199: org.apache.spark.sql.Row\n +- *(1) Filter ((age#168L >= 13) && (age#168L <= 19))\n +- *(1) SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).name, true, false) AS name#167, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).age AS age#168L]\n +- Scan[obj#166]\n","sparkPlanInfo":{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [encodeusingserializer(input[0, java.lang.Object, true], true) AS value#201]","children":[{"nodeName":"MapElements","simpleString":"MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2542/1591023561@7dd320, obj#200: scala.collection.immutable.Map","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject createexternalrow(name#167.toString, age#168L, StructField(name,StringType,true), StructField(age,LongType,false)), obj#199: org.apache.spark.sql.Row","children":[{"nodeName":"Filter","simpleString":"Filter ((age#168L >= 13) && (age#168L <= 19))","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).name, true, false) AS name#167, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).age AS age#168L]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Scan","simpleString":"Scan[obj#166]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":735,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":734,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration total (min, med, max)","accumulatorId":733,"metricType":"timing"}]},"time":1547788475673} +{"Event":"SparkListenerJobStart","Job ID":20,"Submission Time":1547788475754,"Stage Infos":[{"Stage ID":25,"Stage Attempt ID":0,"Stage Name":"collect at SparkSQLExample.scala:230","Number of Tasks":2,"RDD Info":[{"RDD ID":63,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"162\",\"name\":\"mapPartitionsInternal\"}","Callsite":"collect at SparkSQLExample.scala:230","Parent IDs":[62],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"118\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:200","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":62,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"154\",\"name\":\"WholeStageCodegen\"}","Callsite":"collect at SparkSQLExample.scala:230","Parent IDs":[61],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":61,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"161\",\"name\":\"Scan\"}","Callsite":"collect at SparkSQLExample.scala:230","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"117\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:199","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.collect(Dataset.scala:2708)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:230)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[25],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.rdd.scope":"{\"id\":\"163\",\"name\":\"collect\"}","spark.rdd.scope.noOverride":"true","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"16","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":25,"Stage Attempt ID":0,"Stage Name":"collect at SparkSQLExample.scala:230","Number of Tasks":2,"RDD Info":[{"RDD ID":63,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"162\",\"name\":\"mapPartitionsInternal\"}","Callsite":"collect at SparkSQLExample.scala:230","Parent IDs":[62],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"118\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:200","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":62,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"154\",\"name\":\"WholeStageCodegen\"}","Callsite":"collect at SparkSQLExample.scala:230","Parent IDs":[61],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":61,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"161\",\"name\":\"Scan\"}","Callsite":"collect at SparkSQLExample.scala:230","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"117\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:199","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.collect(Dataset.scala:2708)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:230)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788475755,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.rdd.scope":"{\"id\":\"163\",\"name\":\"collect\"}","spark.rdd.scope.noOverride":"true","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"16","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerTaskStart","Stage ID":25,"Stage Attempt ID":0,"Task Info":{"Task ID":217,"Index":0,"Attempt":0,"Launch Time":1547788475761,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":25,"Stage Attempt ID":0,"Task Info":{"Task ID":218,"Index":1,"Attempt":0,"Launch Time":1547788475867,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":25,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":217,"Index":0,"Attempt":0,"Launch Time":1547788475761,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788475868,"Failed":false,"Killed":false,"Accumulables":[{"ID":733,"Name":"duration total (min, med, max)","Update":"6","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":735,"Name":"number of output rows","Update":"2","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":758,"Name":"internal.metrics.input.recordsRead","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":757,"Name":"internal.metrics.input.bytesRead","Update":32,"Value":32,"Internal":true,"Count Failed Values":true},{"ID":740,"Name":"internal.metrics.resultSize","Update":1390,"Value":1390,"Internal":true,"Count Failed Values":true},{"ID":739,"Name":"internal.metrics.executorCpuTime","Update":78410986,"Value":78410986,"Internal":true,"Count Failed Values":true},{"ID":738,"Name":"internal.metrics.executorRunTime","Update":86,"Value":86,"Internal":true,"Count Failed Values":true},{"ID":737,"Name":"internal.metrics.executorDeserializeCpuTime","Update":9743696,"Value":9743696,"Internal":true,"Count Failed Values":true},{"ID":736,"Name":"internal.metrics.executorDeserializeTime","Update":14,"Value":14,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":14,"Executor Deserialize CPU Time":9743696,"Executor Run Time":86,"Executor CPU Time":78410986,"Result Size":1390,"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":32,"Records Read":2},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":25,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":218,"Index":1,"Attempt":0,"Launch Time":1547788475867,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788476471,"Failed":false,"Killed":false,"Accumulables":[{"ID":733,"Name":"duration total (min, med, max)","Update":"582","Value":"587","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":734,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":735,"Name":"number of output rows","Update":"1","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":758,"Name":"internal.metrics.input.recordsRead","Update":1,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":757,"Name":"internal.metrics.input.bytesRead","Update":16,"Value":48,"Internal":true,"Count Failed Values":true},{"ID":740,"Name":"internal.metrics.resultSize","Update":1439,"Value":2829,"Internal":true,"Count Failed Values":true},{"ID":739,"Name":"internal.metrics.executorCpuTime","Update":575036156,"Value":653447142,"Internal":true,"Count Failed Values":true},{"ID":738,"Name":"internal.metrics.executorRunTime","Update":591,"Value":677,"Internal":true,"Count Failed Values":true},{"ID":737,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3688735,"Value":13432431,"Internal":true,"Count Failed Values":true},{"ID":736,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":19,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":3688735,"Executor Run Time":591,"Executor CPU Time":575036156,"Result Size":1439,"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":16,"Records Read":1},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":25,"Stage Attempt ID":0,"Stage Name":"collect at SparkSQLExample.scala:230","Number of Tasks":2,"RDD Info":[{"RDD ID":63,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"162\",\"name\":\"mapPartitionsInternal\"}","Callsite":"collect at SparkSQLExample.scala:230","Parent IDs":[62],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"118\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:200","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":62,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"154\",\"name\":\"WholeStageCodegen\"}","Callsite":"collect at SparkSQLExample.scala:230","Parent IDs":[61],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":61,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"161\",\"name\":\"Scan\"}","Callsite":"collect at SparkSQLExample.scala:230","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"117\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:199","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.collect(Dataset.scala:2708)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:230)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788475755,"Completion Time":1547788476473,"Accumulables":[{"ID":738,"Name":"internal.metrics.executorRunTime","Value":677,"Internal":true,"Count Failed Values":true},{"ID":735,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":737,"Name":"internal.metrics.executorDeserializeCpuTime","Value":13432431,"Internal":true,"Count Failed Values":true},{"ID":758,"Name":"internal.metrics.input.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":740,"Name":"internal.metrics.resultSize","Value":2829,"Internal":true,"Count Failed Values":true},{"ID":734,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":757,"Name":"internal.metrics.input.bytesRead","Value":48,"Internal":true,"Count Failed Values":true},{"ID":733,"Name":"duration total (min, med, max)","Value":"587","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":736,"Name":"internal.metrics.executorDeserializeTime","Value":19,"Internal":true,"Count Failed Values":true},{"ID":739,"Name":"internal.metrics.executorCpuTime","Value":653447142,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":20,"Completion Time":1547788476474,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":16,"time":1547788476729} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":17,"description":"createOrReplaceTempView at SparkSQLExample.scala:258","details":"org.apache.spark.sql.Dataset.createOrReplaceTempView(Dataset.scala:3021)\norg.apache.spark.examples.sql.SparkSQLExample$.runProgrammaticSchemaExample(SparkSQLExample.scala:258)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:52)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\nCreateViewCommand `people`, false, true, LocalTempView\n +- LogicalRDD [name#206, age#207], false\n\n== Analyzed Logical Plan ==\n\nCreateViewCommand `people`, false, true, LocalTempView\n +- LogicalRDD [name#206, age#207], false\n\n== Optimized Logical Plan ==\nCreateViewCommand `people`, false, true, LocalTempView\n +- LogicalRDD [name#206, age#207], false\n\n== Physical Plan ==\nExecute CreateViewCommand\n +- CreateViewCommand `people`, false, true, LocalTempView\n +- LogicalRDD [name#206, age#207], false\n","sparkPlanInfo":{"nodeName":"Execute CreateViewCommand","simpleString":"Execute CreateViewCommand","children":[],"metadata":{},"metrics":[]},"time":1547788476801} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":17,"time":1547788476802} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":18,"description":"show at SparkSQLExample.scala:265","details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runProgrammaticSchemaExample(SparkSQLExample.scala:265)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:52)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(value#214 as string) AS value#218]\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#214]\n +- MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2583/185651170@6b9d8f7, interface org.apache.spark.sql.Row, [StructField(name,StringType,true)], obj#213: java.lang.String\n +- DeserializeToObject createexternalrow(name#206.toString, StructField(name,StringType,true)), obj#212: org.apache.spark.sql.Row\n +- Project [name#206]\n +- SubqueryAlias `people`\n +- LogicalRDD [name#206, age#207], false\n\n== Analyzed Logical Plan ==\nvalue: string\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(value#214 as string) AS value#218]\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#214]\n +- MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2583/185651170@6b9d8f7, interface org.apache.spark.sql.Row, [StructField(name,StringType,true)], obj#213: java.lang.String\n +- DeserializeToObject createexternalrow(name#206.toString, StructField(name,StringType,true)), obj#212: org.apache.spark.sql.Row\n +- Project [name#206]\n +- SubqueryAlias `people`\n +- LogicalRDD [name#206, age#207], false\n\n== Optimized Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#214]\n +- MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2583/185651170@6b9d8f7, interface org.apache.spark.sql.Row, [StructField(name,StringType,true)], obj#213: java.lang.String\n +- DeserializeToObject createexternalrow(name#206.toString, StructField(name,StringType,true)), obj#212: org.apache.spark.sql.Row\n +- Project [name#206]\n +- LogicalRDD [name#206, age#207], false\n\n== Physical Plan ==\nCollectLimit 21\n+- *(1) SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#214]\n +- *(1) MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2583/185651170@6b9d8f7, obj#213: java.lang.String\n +- *(1) DeserializeToObject createexternalrow(name#206.toString, StructField(name,StringType,true)), obj#212: org.apache.spark.sql.Row\n +- *(1) Project [name#206]\n +- *(1) Scan ExistingRDD[name#206,age#207]\n","sparkPlanInfo":{"nodeName":"CollectLimit","simpleString":"CollectLimit 21","children":[{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#214]","children":[{"nodeName":"MapElements","simpleString":"MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2583/185651170@6b9d8f7, obj#213: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject createexternalrow(name#206.toString, StructField(name,StringType,true)), obj#212: org.apache.spark.sql.Row","children":[{"nodeName":"Project","simpleString":"Project [name#206]","children":[{"nodeName":"Scan ExistingRDD","simpleString":"Scan ExistingRDD[name#206,age#207]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":772,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration total (min, med, max)","accumulatorId":771,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":769,"metricType":"sum"},{"name":"shuffle write time total (min, med, max)","accumulatorId":770,"metricType":"nsTiming"},{"name":"records read","accumulatorId":767,"metricType":"sum"},{"name":"local bytes read total (min, med, max)","accumulatorId":765,"metricType":"size"},{"name":"fetch wait time total (min, med, max)","accumulatorId":766,"metricType":"timing"},{"name":"remote bytes read total (min, med, max)","accumulatorId":763,"metricType":"size"},{"name":"local blocks read","accumulatorId":762,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":761,"metricType":"sum"},{"name":"remote bytes read to disk total (min, med, max)","accumulatorId":764,"metricType":"size"},{"name":"shuffle bytes written total (min, med, max)","accumulatorId":768,"metricType":"size"}]},"time":1547788476854} +{"Event":"SparkListenerJobStart","Job ID":21,"Submission Time":1547788476899,"Stage Infos":[{"Stage ID":26,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:265","Number of Tasks":1,"RDD Info":[{"RDD ID":71,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"179\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[70],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":69,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"172\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[68],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":70,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"178\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[69],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":65,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"168\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:239","Parent IDs":[64],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":67,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"170\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:252","Parent IDs":[66],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":64,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"168\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:239","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":66,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"169\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:251","Parent IDs":[65],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":68,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"171\",\"name\":\"map\"}","Callsite":"createDataFrame at SparkSQLExample.scala:255","Parent IDs":[67],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runProgrammaticSchemaExample(SparkSQLExample.scala:265)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:52)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[26],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"18","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":26,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:265","Number of Tasks":1,"RDD Info":[{"RDD ID":71,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"179\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[70],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":69,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"172\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[68],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":70,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"178\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[69],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":65,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"168\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:239","Parent IDs":[64],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":67,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"170\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:252","Parent IDs":[66],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":64,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"168\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:239","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":66,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"169\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:251","Parent IDs":[65],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":68,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"171\",\"name\":\"map\"}","Callsite":"createDataFrame at SparkSQLExample.scala:255","Parent IDs":[67],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runProgrammaticSchemaExample(SparkSQLExample.scala:265)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:52)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788476901,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"18","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerTaskStart","Stage ID":26,"Stage Attempt ID":0,"Task Info":{"Task ID":219,"Index":0,"Attempt":0,"Launch Time":1547788476909,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":26,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":219,"Index":0,"Attempt":0,"Launch Time":1547788476909,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788479653,"Failed":false,"Killed":false,"Accumulables":[{"ID":771,"Name":"duration total (min, med, max)","Update":"2512","Value":"2511","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":772,"Name":"number of output rows","Update":"2","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":795,"Name":"internal.metrics.input.recordsRead","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":794,"Name":"internal.metrics.input.bytesRead","Update":32,"Value":32,"Internal":true,"Count Failed Values":true},{"ID":778,"Name":"internal.metrics.jvmGCTime","Update":37,"Value":37,"Internal":true,"Count Failed Values":true},{"ID":777,"Name":"internal.metrics.resultSize","Update":1240,"Value":1240,"Internal":true,"Count Failed Values":true},{"ID":776,"Name":"internal.metrics.executorCpuTime","Update":2467792536,"Value":2467792536,"Internal":true,"Count Failed Values":true},{"ID":775,"Name":"internal.metrics.executorRunTime","Update":2594,"Value":2594,"Internal":true,"Count Failed Values":true},{"ID":774,"Name":"internal.metrics.executorDeserializeCpuTime","Update":136342693,"Value":136342693,"Internal":true,"Count Failed Values":true},{"ID":773,"Name":"internal.metrics.executorDeserializeTime","Update":144,"Value":144,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":144,"Executor Deserialize CPU Time":136342693,"Executor Run Time":2594,"Executor CPU Time":2467792536,"Result Size":1240,"JVM GC Time":37,"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":32,"Records Read":2},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":26,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:265","Number of Tasks":1,"RDD Info":[{"RDD ID":71,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"179\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[70],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":69,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"172\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[68],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":70,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"178\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[69],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":65,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"168\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:239","Parent IDs":[64],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":67,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"170\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:252","Parent IDs":[66],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":64,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"168\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:239","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":66,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"169\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:251","Parent IDs":[65],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":68,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"171\",\"name\":\"map\"}","Callsite":"createDataFrame at SparkSQLExample.scala:255","Parent IDs":[67],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runProgrammaticSchemaExample(SparkSQLExample.scala:265)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:52)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788476901,"Completion Time":1547788479654,"Accumulables":[{"ID":774,"Name":"internal.metrics.executorDeserializeCpuTime","Value":136342693,"Internal":true,"Count Failed Values":true},{"ID":795,"Name":"internal.metrics.input.recordsRead","Value":2,"Internal":true,"Count Failed Values":true},{"ID":777,"Name":"internal.metrics.resultSize","Value":1240,"Internal":true,"Count Failed Values":true},{"ID":771,"Name":"duration total (min, med, max)","Value":"2511","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":773,"Name":"internal.metrics.executorDeserializeTime","Value":144,"Internal":true,"Count Failed Values":true},{"ID":794,"Name":"internal.metrics.input.bytesRead","Value":32,"Internal":true,"Count Failed Values":true},{"ID":776,"Name":"internal.metrics.executorCpuTime","Value":2467792536,"Internal":true,"Count Failed Values":true},{"ID":775,"Name":"internal.metrics.executorRunTime","Value":2594,"Internal":true,"Count Failed Values":true},{"ID":778,"Name":"internal.metrics.jvmGCTime","Value":37,"Internal":true,"Count Failed Values":true},{"ID":772,"Name":"number of output rows","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"}]}} +{"Event":"SparkListenerJobEnd","Job ID":21,"Completion Time":1547788479655,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":22,"Submission Time":1547788479665,"Stage Infos":[{"Stage ID":27,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:265","Number of Tasks":1,"RDD Info":[{"RDD ID":71,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"179\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[70],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":69,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"172\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[68],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":70,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"178\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[69],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":65,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"168\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:239","Parent IDs":[64],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":67,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"170\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:252","Parent IDs":[66],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":64,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"168\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:239","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":66,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"169\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:251","Parent IDs":[65],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":68,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"171\",\"name\":\"map\"}","Callsite":"createDataFrame at SparkSQLExample.scala:255","Parent IDs":[67],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runProgrammaticSchemaExample(SparkSQLExample.scala:265)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:52)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[27],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"18","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":27,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:265","Number of Tasks":1,"RDD Info":[{"RDD ID":71,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"179\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[70],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":69,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"172\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[68],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":70,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"178\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[69],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":65,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"168\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:239","Parent IDs":[64],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":67,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"170\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:252","Parent IDs":[66],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":64,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"168\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:239","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":66,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"169\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:251","Parent IDs":[65],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":68,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"171\",\"name\":\"map\"}","Callsite":"createDataFrame at SparkSQLExample.scala:255","Parent IDs":[67],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runProgrammaticSchemaExample(SparkSQLExample.scala:265)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:52)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788479666,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"18","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerTaskStart","Stage ID":27,"Stage Attempt ID":0,"Task Info":{"Task ID":220,"Index":0,"Attempt":0,"Launch Time":1547788479676,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":27,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":220,"Index":0,"Attempt":0,"Launch Time":1547788479676,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788479710,"Failed":false,"Killed":false,"Accumulables":[{"ID":771,"Name":"duration total (min, med, max)","Update":"6","Value":"2517","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":772,"Name":"number of output rows","Update":"1","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":820,"Name":"internal.metrics.input.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":819,"Name":"internal.metrics.input.bytesRead","Update":16,"Value":16,"Internal":true,"Count Failed Values":true},{"ID":802,"Name":"internal.metrics.resultSize","Update":1180,"Value":1180,"Internal":true,"Count Failed Values":true},{"ID":801,"Name":"internal.metrics.executorCpuTime","Update":12086508,"Value":12086508,"Internal":true,"Count Failed Values":true},{"ID":800,"Name":"internal.metrics.executorRunTime","Update":14,"Value":14,"Internal":true,"Count Failed Values":true},{"ID":799,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8317054,"Value":8317054,"Internal":true,"Count Failed Values":true},{"ID":798,"Name":"internal.metrics.executorDeserializeTime","Update":14,"Value":14,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":14,"Executor Deserialize CPU Time":8317054,"Executor Run Time":14,"Executor CPU Time":12086508,"Result Size":1180,"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":16,"Records Read":1},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":27,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:265","Number of Tasks":1,"RDD Info":[{"RDD ID":71,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"179\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[70],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":69,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"172\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[68],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":70,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"178\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[69],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":65,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"168\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:239","Parent IDs":[64],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":67,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"170\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:252","Parent IDs":[66],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":64,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"168\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:239","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":66,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"169\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:251","Parent IDs":[65],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":68,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"171\",\"name\":\"map\"}","Callsite":"createDataFrame at SparkSQLExample.scala:255","Parent IDs":[67],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runProgrammaticSchemaExample(SparkSQLExample.scala:265)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:52)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788479666,"Completion Time":1547788479711,"Accumulables":[{"ID":819,"Name":"internal.metrics.input.bytesRead","Value":16,"Internal":true,"Count Failed Values":true},{"ID":801,"Name":"internal.metrics.executorCpuTime","Value":12086508,"Internal":true,"Count Failed Values":true},{"ID":771,"Name":"duration total (min, med, max)","Value":"2517","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":800,"Name":"internal.metrics.executorRunTime","Value":14,"Internal":true,"Count Failed Values":true},{"ID":820,"Name":"internal.metrics.input.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":802,"Name":"internal.metrics.resultSize","Value":1180,"Internal":true,"Count Failed Values":true},{"ID":772,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":799,"Name":"internal.metrics.executorDeserializeCpuTime","Value":8317054,"Internal":true,"Count Failed Values":true},{"ID":798,"Name":"internal.metrics.executorDeserializeTime","Value":14,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":22,"Completion Time":1547788479711,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":18,"time":1547788479712} +{"Event":"SparkListenerApplicationEnd","Timestamp":1547788479714} 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 1a071fa771330..1ae529150becd 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 @@ -183,19 +183,9 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers // in the test resource folder cases.foreach { case (name, path) => test(name) { - val (code, jsonOpt, errOpt) = getContentAndCode(path) - code should be (HttpServletResponse.SC_OK) - jsonOpt should be ('defined) - errOpt should be (None) - - val exp = IOUtils.toString(new FileInputStream( - new File(expRoot, HistoryServerSuite.sanitizePath(name) + "_expectation.json"))) - // compare the ASTs so formatting differences don't cause failures - import org.json4s._ - import org.json4s.jackson.JsonMethods._ - val jsonAst = parse(clearLastUpdated(jsonOpt.get)) - val expAst = parse(exp) - assertValidDataInJson(jsonAst, expAst) + val expectationFile = new File(expRoot, HistoryServerSuite.sanitizePath(name) + + "_expectation.json") + assertApiCallResponse(path, expectationFile) } } @@ -642,6 +632,38 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers } } + test("Apply custom log urls") { + // This only verifies whether applying custom log URLs is in effect for SHS. + // Validation of "custom log URLs" functionality will be covered from different UTs. + + // restart server + try { + stop() + init((History.CUSTOM_EXECUTOR_LOG_URL.key, "http://newhost:9999/logs/clusters/" + + "{{CLUSTER_ID}}/users/{{USER}}/containers/{{CONTAINER_ID}}/{{FILE_NAME}})")) + + val path = "applications/application_1547723113049_0005/1/executors" + val expectation = "executor_list_json_apply_custom_log_urls_expectation.json" + + val (code, jsonOpt, errOpt) = getContentAndCode(path) + code should be (HttpServletResponse.SC_OK) + jsonOpt should be ('defined) + errOpt should be (None) + + val exp = IOUtils.toString(new FileInputStream(new File(expRoot, expectation))) + // compare the ASTs so formatting differences don't cause failures + import org.json4s._ + import org.json4s.jackson.JsonMethods._ + val jsonAst = parse(clearLastUpdated(jsonOpt.get)) + val expAst = parse(exp) + assertValidDataInJson(jsonAst, expAst) + + } finally { + // make sure other UTs are not affected from relaunching HistoryServer + stop() + } + } + def getContentAndCode(path: String, port: Int = port): (Int, Option[String], Option[String]) = { HistoryServerSuite.getContentAndCode(new URL(s"http://localhost:$port/api/v1/$path")) } @@ -663,6 +685,21 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers out.close() } + private def assertApiCallResponse(path: String, expectationFile: File): Unit = { + val (code, jsonOpt, errOpt) = getContentAndCode(path) + code should be (HttpServletResponse.SC_OK) + jsonOpt should be ('defined) + errOpt should be (None) + + val exp = IOUtils.toString(new FileInputStream(expectationFile)) + // compare the ASTs so formatting differences don't cause failures + import org.json4s._ + import org.json4s.jackson.JsonMethods._ + val jsonAst = parse(clearLastUpdated(jsonOpt.get)) + val expAst = parse(exp) + assertValidDataInJson(jsonAst, expAst) + } + } object HistoryServerSuite { From e3d3d498d1108ebd42fa350d0fd213907cc39136 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Fri, 18 Jan 2019 22:01:17 +0900 Subject: [PATCH 10/28] Add missing RAT exclude, as well as code cleaning a bit --- .../spark/deploy/history/HistoryServerSuite.scala | 14 +------------- dev/.rat-excludes | 1 + 2 files changed, 2 insertions(+), 13 deletions(-) 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 1ae529150becd..f3e7d761dadeb 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 @@ -645,19 +645,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers val path = "applications/application_1547723113049_0005/1/executors" val expectation = "executor_list_json_apply_custom_log_urls_expectation.json" - val (code, jsonOpt, errOpt) = getContentAndCode(path) - code should be (HttpServletResponse.SC_OK) - jsonOpt should be ('defined) - errOpt should be (None) - - val exp = IOUtils.toString(new FileInputStream(new File(expRoot, expectation))) - // compare the ASTs so formatting differences don't cause failures - import org.json4s._ - import org.json4s.jackson.JsonMethods._ - val jsonAst = parse(clearLastUpdated(jsonOpt.get)) - val expAst = parse(exp) - assertValidDataInJson(jsonAst, expAst) - + assertApiCallResponse(path, new File(expRoot, expectation)) } finally { // make sure other UTs are not affected from relaunching HistoryServer stop() diff --git a/dev/.rat-excludes b/dev/.rat-excludes index 8239cbc3a381c..93f86b1ee390f 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_1547723113049_0005 stat local-1422981759269 local-1422981780767 From 08bf035fc92afd452e744dca3c1ba3ad12dc34a9 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Fri, 18 Jan 2019 22:21:48 +0900 Subject: [PATCH 11/28] Reduce lines of new test case file --- ...son_apply_custom_log_urls_expectation.json | 22 +- .../application_1547723113049_0005 | 577 ------------------ 2 files changed, 11 insertions(+), 588 deletions(-) diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_json_apply_custom_log_urls_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_json_apply_custom_log_urls_expectation.json index 61ba87e4c656d..4494767a45a63 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_list_json_apply_custom_log_urls_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_list_json_apply_custom_log_urls_expectation.json @@ -48,10 +48,10 @@ "maxTasks":1, "activeTasks":0, "failedTasks":0, - "completedTasks":70, - "totalTasks":70, - "totalDuration":6250, - "totalGCTime":162, + "completedTasks":0, + "totalTasks":0, + "totalDuration":0, + "totalGCTime":0, "totalInputBytes":0, "totalShuffleRead":0, "totalShuffleWrite":0, @@ -91,13 +91,13 @@ "maxTasks":1, "activeTasks":0, "failedTasks":0, - "completedTasks":151, - "totalTasks":151, - "totalDuration":13343, - "totalGCTime":264, - "totalInputBytes":995, - "totalShuffleRead":186, - "totalShuffleWrite":186, + "completedTasks":1, + "totalTasks":1, + "totalDuration":4025, + "totalGCTime":159, + "totalInputBytes":73, + "totalShuffleRead":0, + "totalShuffleWrite":0, "isBlacklisted":false, "maxMemory":384093388, "addTime":"2019-01-18T05:14:14.209GMT", diff --git a/core/src/test/resources/spark-events/application_1547723113049_0005 b/core/src/test/resources/spark-events/application_1547723113049_0005 index 718a7e6e0fe00..a409762608a47 100644 --- a/core/src/test/resources/spark-events/application_1547723113049_0005 +++ b/core/src/test/resources/spark-events/application_1547723113049_0005 @@ -12,581 +12,4 @@ {"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":1547788459627,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788463652,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"duration total (min, med, max)","Update":"1582","Value":"1581","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":0,"Name":"number of output rows","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"internal.metrics.input.recordsRead","Update":3,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":26,"Name":"internal.metrics.input.bytesRead","Update":73,"Value":73,"Internal":true,"Count Failed Values":true},{"ID":11,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.jvmGCTime","Update":159,"Value":159,"Internal":true,"Count Failed Values":true},{"ID":9,"Name":"internal.metrics.resultSize","Update":1963,"Value":1963,"Internal":true,"Count Failed Values":true},{"ID":8,"Name":"internal.metrics.executorCpuTime","Update":2225128499,"Value":2225128499,"Internal":true,"Count Failed Values":true},{"ID":7,"Name":"internal.metrics.executorRunTime","Update":2291,"Value":2291,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1171850277,"Value":1171850277,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.executorDeserializeTime","Update":1431,"Value":1431,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1431,"Executor Deserialize CPU Time":1171850277,"Executor Run Time":2291,"Executor CPU Time":2225128499,"Result Size":1963,"JVM GC Time":159,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":73,"Records Read":3},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"json at SparkSQLExample.scala:59","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"mapPartitions\"}","Callsite":"json at SparkSQLExample.scala:59","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"json at SparkSQLExample.scala:59","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"json at SparkSQLExample.scala:59","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameReader.json(DataFrameReader.scala:331)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:59)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788459321,"Completion Time":1547788463667,"Accumulables":[{"ID":26,"Name":"internal.metrics.input.bytesRead","Value":73,"Internal":true,"Count Failed Values":true},{"ID":8,"Name":"internal.metrics.executorCpuTime","Value":2225128499,"Internal":true,"Count Failed Values":true},{"ID":11,"Name":"internal.metrics.resultSerializationTime","Value":2,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.executorDeserializeTime","Value":1431,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"duration total (min, med, max)","Value":"1581","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"internal.metrics.executorRunTime","Value":2291,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.jvmGCTime","Value":159,"Internal":true,"Count Failed Values":true},{"ID":27,"Name":"internal.metrics.input.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":9,"Name":"internal.metrics.resultSize","Value":1963,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.executorDeserializeCpuTime","Value":1171850277,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"}]}} {"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1547788463673,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":0,"description":"show at SparkSQLExample.scala:62","details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:62)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#7L as string) AS age#13, cast(name#8 as string) AS name#14]\n +- Relation[age#7L,name#8] json\n\n== Analyzed Logical Plan ==\nage: string, name: string\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#7L as string) AS age#13, cast(name#8 as string) AS name#14]\n +- Relation[age#7L,name#8] json\n\n== Optimized Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#7L as string) AS age#13, name#8]\n +- Relation[age#7L,name#8] json\n\n== Physical Plan ==\nCollectLimit 21\n+- *(1) Project [cast(age#7L as string) AS age#13, name#8]\n +- *(1) FileScan json [age#7L,name#8] Batched: false, DataFilters: [], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n","sparkPlanInfo":{"nodeName":"CollectLimit","simpleString":"CollectLimit 21","children":[{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"Project","simpleString":"Project [cast(age#7L as string) AS age#13, name#8]","children":[{"nodeName":"Scan json ","simpleString":"FileScan json [age#7L,name#8] Batched: false, DataFilters: [], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct","children":[],"metadata":{"Location":"InMemoryFileIndex[hdfs://node6:8020/user/spark/examples/src/main/resources/people.json]","ReadSchema":"struct","Format":"JSON","Batched":"false","PartitionFilters":"[]","PushedFilters":"[]","DataFilters":"[]"},"metrics":[{"name":"number of output rows","accumulatorId":41,"metricType":"sum"},{"name":"number of files","accumulatorId":42,"metricType":"sum"},{"name":"metadata time","accumulatorId":43,"metricType":"sum"},{"name":"scan time total (min, med, max)","accumulatorId":44,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration total (min, med, max)","accumulatorId":40,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":38,"metricType":"sum"},{"name":"shuffle write time total (min, med, max)","accumulatorId":39,"metricType":"nsTiming"},{"name":"records read","accumulatorId":36,"metricType":"sum"},{"name":"local bytes read total (min, med, max)","accumulatorId":34,"metricType":"size"},{"name":"fetch wait time total (min, med, max)","accumulatorId":35,"metricType":"timing"},{"name":"remote bytes read total (min, med, max)","accumulatorId":32,"metricType":"size"},{"name":"local blocks read","accumulatorId":31,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":30,"metricType":"sum"},{"name":"remote bytes read to disk total (min, med, max)","accumulatorId":33,"metricType":"size"},{"name":"shuffle bytes written total (min, med, max)","accumulatorId":37,"metricType":"size"}]},"time":1547788463790} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":0,"accumUpdates":[[42,1],[43,0]]} -{"Event":"SparkListenerJobStart","Job ID":1,"Submission Time":1547788463962,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:62","Number of Tasks":1,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:62","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"FileScanRDD","Scope":"{\"id\":\"6\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:62","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:62","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"6\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:62","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:62)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[1],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"0","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:62","Number of Tasks":1,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:62","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"FileScanRDD","Scope":"{\"id\":\"6\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:62","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:62","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"6\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:62","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:62)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788463966,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"0","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":0,"Attempt":0,"Launch Time":1547788463977,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":0,"Attempt":0,"Launch Time":1547788463977,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788464352,"Failed":false,"Killed":false,"Accumulables":[{"ID":40,"Name":"duration total (min, med, max)","Update":"172","Value":"171","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":41,"Name":"number of output rows","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":67,"Name":"internal.metrics.input.recordsRead","Update":3,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.input.bytesRead","Update":73,"Value":73,"Internal":true,"Count Failed Values":true},{"ID":49,"Name":"internal.metrics.resultSize","Update":1229,"Value":1229,"Internal":true,"Count Failed Values":true},{"ID":48,"Name":"internal.metrics.executorCpuTime","Update":262646529,"Value":262646529,"Internal":true,"Count Failed Values":true},{"ID":47,"Name":"internal.metrics.executorRunTime","Update":283,"Value":283,"Internal":true,"Count Failed Values":true},{"ID":46,"Name":"internal.metrics.executorDeserializeCpuTime","Update":61019763,"Value":61019763,"Internal":true,"Count Failed Values":true},{"ID":45,"Name":"internal.metrics.executorDeserializeTime","Update":73,"Value":73,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":73,"Executor Deserialize CPU Time":61019763,"Executor Run Time":283,"Executor CPU Time":262646529,"Result Size":1229,"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":73,"Records Read":3},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:62","Number of Tasks":1,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"10\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:62","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"FileScanRDD","Scope":"{\"id\":\"6\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:62","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:62","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"6\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:62","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:62)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788463966,"Completion Time":1547788464354,"Accumulables":[{"ID":41,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":47,"Name":"internal.metrics.executorRunTime","Value":283,"Internal":true,"Count Failed Values":true},{"ID":46,"Name":"internal.metrics.executorDeserializeCpuTime","Value":61019763,"Internal":true,"Count Failed Values":true},{"ID":40,"Name":"duration total (min, med, max)","Value":"171","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":67,"Name":"internal.metrics.input.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":49,"Name":"internal.metrics.resultSize","Value":1229,"Internal":true,"Count Failed Values":true},{"ID":45,"Name":"internal.metrics.executorDeserializeTime","Value":73,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.input.bytesRead","Value":73,"Internal":true,"Count Failed Values":true},{"ID":48,"Name":"internal.metrics.executorCpuTime","Value":262646529,"Internal":true,"Count Failed Values":true}]}} -{"Event":"SparkListenerJobEnd","Job ID":1,"Completion Time":1547788464354,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":0,"time":1547788464363} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":1,"description":"show at SparkSQLExample.scala:82","details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:82)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(name#8 as string) AS name#24]\n +- Project [name#8]\n +- Relation[age#7L,name#8] json\n\n== Analyzed Logical Plan ==\nname: string\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(name#8 as string) AS name#24]\n +- Project [name#8]\n +- Relation[age#7L,name#8] json\n\n== Optimized Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [name#8]\n +- Relation[age#7L,name#8] json\n\n== Physical Plan ==\nCollectLimit 21\n+- *(1) FileScan json [name#8] Batched: false, DataFilters: [], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n","sparkPlanInfo":{"nodeName":"CollectLimit","simpleString":"CollectLimit 21","children":[{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"Scan json ","simpleString":"FileScan json [name#8] Batched: false, DataFilters: [], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct","children":[],"metadata":{"Location":"InMemoryFileIndex[hdfs://node6:8020/user/spark/examples/src/main/resources/people.json]","ReadSchema":"struct","Format":"JSON","Batched":"false","PartitionFilters":"[]","PushedFilters":"[]","DataFilters":"[]"},"metrics":[{"name":"number of output rows","accumulatorId":81,"metricType":"sum"},{"name":"number of files","accumulatorId":82,"metricType":"sum"},{"name":"metadata time","accumulatorId":83,"metricType":"sum"},{"name":"scan time total (min, med, max)","accumulatorId":84,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"duration total (min, med, max)","accumulatorId":80,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":78,"metricType":"sum"},{"name":"shuffle write time total (min, med, max)","accumulatorId":79,"metricType":"nsTiming"},{"name":"records read","accumulatorId":76,"metricType":"sum"},{"name":"local bytes read total (min, med, max)","accumulatorId":74,"metricType":"size"},{"name":"fetch wait time total (min, med, max)","accumulatorId":75,"metricType":"timing"},{"name":"remote bytes read total (min, med, max)","accumulatorId":72,"metricType":"size"},{"name":"local blocks read","accumulatorId":71,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":70,"metricType":"sum"},{"name":"remote bytes read to disk total (min, med, max)","accumulatorId":73,"metricType":"size"},{"name":"shuffle bytes written total (min, med, max)","accumulatorId":77,"metricType":"size"}]},"time":1547788464413} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":1,"accumUpdates":[[82,1],[83,0]]} -{"Event":"SparkListenerJobStart","Job ID":2,"Submission Time":1547788464480,"Stage Infos":[{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:82","Number of Tasks":1,"RDD Info":[{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:82","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"17\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:82","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"15\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:82","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"FileScanRDD","Scope":"{\"id\":\"15\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:82","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:82)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[2],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"1","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:82","Number of Tasks":1,"RDD Info":[{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:82","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"17\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:82","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"15\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:82","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"FileScanRDD","Scope":"{\"id\":\"15\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:82","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:82)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788464483,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"1","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":0,"Attempt":0,"Launch Time":1547788464494,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":0,"Attempt":0,"Launch Time":1547788464494,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788464642,"Failed":false,"Killed":false,"Accumulables":[{"ID":80,"Name":"duration total (min, med, max)","Update":"89","Value":"88","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":81,"Name":"number of output rows","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":107,"Name":"internal.metrics.input.recordsRead","Update":3,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.input.bytesRead","Update":73,"Value":73,"Internal":true,"Count Failed Values":true},{"ID":89,"Name":"internal.metrics.resultSize","Update":1203,"Value":1203,"Internal":true,"Count Failed Values":true},{"ID":88,"Name":"internal.metrics.executorCpuTime","Update":80442300,"Value":80442300,"Internal":true,"Count Failed Values":true},{"ID":87,"Name":"internal.metrics.executorRunTime","Update":96,"Value":96,"Internal":true,"Count Failed Values":true},{"ID":86,"Name":"internal.metrics.executorDeserializeCpuTime","Update":24843191,"Value":24843191,"Internal":true,"Count Failed Values":true},{"ID":85,"Name":"internal.metrics.executorDeserializeTime","Update":35,"Value":35,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":35,"Executor Deserialize CPU Time":24843191,"Executor Run Time":96,"Executor CPU Time":80442300,"Result Size":1203,"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":73,"Records Read":3},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:82","Number of Tasks":1,"RDD Info":[{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:82","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"17\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:82","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"15\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:82","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"FileScanRDD","Scope":"{\"id\":\"15\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:82","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:82)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788464483,"Completion Time":1547788464644,"Accumulables":[{"ID":86,"Name":"internal.metrics.executorDeserializeCpuTime","Value":24843191,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"duration total (min, med, max)","Value":"88","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":107,"Name":"internal.metrics.input.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":89,"Name":"internal.metrics.resultSize","Value":1203,"Internal":true,"Count Failed Values":true},{"ID":106,"Name":"internal.metrics.input.bytesRead","Value":73,"Internal":true,"Count Failed Values":true},{"ID":85,"Name":"internal.metrics.executorDeserializeTime","Value":35,"Internal":true,"Count Failed Values":true},{"ID":88,"Name":"internal.metrics.executorCpuTime","Value":80442300,"Internal":true,"Count Failed Values":true},{"ID":87,"Name":"internal.metrics.executorRunTime","Value":96,"Internal":true,"Count Failed Values":true},{"ID":81,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"}]}} -{"Event":"SparkListenerJobEnd","Job ID":2,"Completion Time":1547788464645,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":1,"time":1547788464646} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":2,"description":"show at SparkSQLExample.scala:92","details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:92)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(name#8 as string) AS name#34, cast((age + 1)#29L as string) AS (age + 1)#35]\n +- Project [name#8, (age#7L + cast(1 as bigint)) AS (age + 1)#29L]\n +- Relation[age#7L,name#8] json\n\n== Analyzed Logical Plan ==\nname: string, (age + 1): string\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(name#8 as string) AS name#34, cast((age + 1)#29L as string) AS (age + 1)#35]\n +- Project [name#8, (age#7L + cast(1 as bigint)) AS (age + 1)#29L]\n +- Relation[age#7L,name#8] json\n\n== Optimized Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [name#8, cast((age#7L + 1) as string) AS (age + 1)#35]\n +- Relation[age#7L,name#8] json\n\n== Physical Plan ==\nCollectLimit 21\n+- *(1) Project [name#8, cast((age#7L + 1) as string) AS (age + 1)#35]\n +- *(1) FileScan json [age#7L,name#8] Batched: false, DataFilters: [], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n","sparkPlanInfo":{"nodeName":"CollectLimit","simpleString":"CollectLimit 21","children":[{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"Project","simpleString":"Project [name#8, cast((age#7L + 1) as string) AS (age + 1)#35]","children":[{"nodeName":"Scan json ","simpleString":"FileScan json [age#7L,name#8] Batched: false, DataFilters: [], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct","children":[],"metadata":{"Location":"InMemoryFileIndex[hdfs://node6:8020/user/spark/examples/src/main/resources/people.json]","ReadSchema":"struct","Format":"JSON","Batched":"false","PartitionFilters":"[]","PushedFilters":"[]","DataFilters":"[]"},"metrics":[{"name":"number of output rows","accumulatorId":121,"metricType":"sum"},{"name":"number of files","accumulatorId":122,"metricType":"sum"},{"name":"metadata time","accumulatorId":123,"metricType":"sum"},{"name":"scan time total (min, med, max)","accumulatorId":124,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration total (min, med, max)","accumulatorId":120,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":118,"metricType":"sum"},{"name":"shuffle write time total (min, med, max)","accumulatorId":119,"metricType":"nsTiming"},{"name":"records read","accumulatorId":116,"metricType":"sum"},{"name":"local bytes read total (min, med, max)","accumulatorId":114,"metricType":"size"},{"name":"fetch wait time total (min, med, max)","accumulatorId":115,"metricType":"timing"},{"name":"remote bytes read total (min, med, max)","accumulatorId":112,"metricType":"size"},{"name":"local blocks read","accumulatorId":111,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":110,"metricType":"sum"},{"name":"remote bytes read to disk total (min, med, max)","accumulatorId":113,"metricType":"size"},{"name":"shuffle bytes written total (min, med, max)","accumulatorId":117,"metricType":"size"}]},"time":1547788464810} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":2,"accumUpdates":[[122,1],[123,3]]} -{"Event":"SparkListenerJobStart","Job ID":3,"Submission Time":1547788464899,"Stage Infos":[{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:92","Number of Tasks":1,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"27\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:92","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"FileScanRDD","Scope":"{\"id\":\"23\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:92","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"23\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:92","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"26\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:92","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:92)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[3],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"2","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:92","Number of Tasks":1,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"27\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:92","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"FileScanRDD","Scope":"{\"id\":\"23\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:92","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"23\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:92","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"26\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:92","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:92)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788464902,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"2","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":0,"Attempt":0,"Launch Time":1547788464910,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":0,"Attempt":0,"Launch Time":1547788464910,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788465075,"Failed":false,"Killed":false,"Accumulables":[{"ID":120,"Name":"duration total (min, med, max)","Update":"86","Value":"85","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":121,"Name":"number of output rows","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":147,"Name":"internal.metrics.input.recordsRead","Update":3,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":146,"Name":"internal.metrics.input.bytesRead","Update":73,"Value":73,"Internal":true,"Count Failed Values":true},{"ID":129,"Name":"internal.metrics.resultSize","Update":1234,"Value":1234,"Internal":true,"Count Failed Values":true},{"ID":128,"Name":"internal.metrics.executorCpuTime","Update":109502826,"Value":109502826,"Internal":true,"Count Failed Values":true},{"ID":127,"Name":"internal.metrics.executorRunTime","Update":127,"Value":127,"Internal":true,"Count Failed Values":true},{"ID":126,"Name":"internal.metrics.executorDeserializeCpuTime","Update":17361720,"Value":17361720,"Internal":true,"Count Failed Values":true},{"ID":125,"Name":"internal.metrics.executorDeserializeTime","Update":24,"Value":24,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":24,"Executor Deserialize CPU Time":17361720,"Executor Run Time":127,"Executor CPU Time":109502826,"Result Size":1234,"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":73,"Records Read":3},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:92","Number of Tasks":1,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"27\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:92","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"FileScanRDD","Scope":"{\"id\":\"23\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:92","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"23\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:92","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"26\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:92","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:92)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788464902,"Completion Time":1547788465077,"Accumulables":[{"ID":146,"Name":"internal.metrics.input.bytesRead","Value":73,"Internal":true,"Count Failed Values":true},{"ID":128,"Name":"internal.metrics.executorCpuTime","Value":109502826,"Internal":true,"Count Failed Values":true},{"ID":125,"Name":"internal.metrics.executorDeserializeTime","Value":24,"Internal":true,"Count Failed Values":true},{"ID":127,"Name":"internal.metrics.executorRunTime","Value":127,"Internal":true,"Count Failed Values":true},{"ID":121,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":120,"Name":"duration total (min, med, max)","Value":"85","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":147,"Name":"internal.metrics.input.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":129,"Name":"internal.metrics.resultSize","Value":1234,"Internal":true,"Count Failed Values":true},{"ID":126,"Name":"internal.metrics.executorDeserializeCpuTime","Value":17361720,"Internal":true,"Count Failed Values":true}]}} -{"Event":"SparkListenerJobEnd","Job ID":3,"Completion Time":1547788465078,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":2,"time":1547788465079} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":3,"description":"show at SparkSQLExample.scala:102","details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:102)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#7L as string) AS age#45, cast(name#8 as string) AS name#46]\n +- Filter (age#7L > cast(21 as bigint))\n +- Relation[age#7L,name#8] json\n\n== Analyzed Logical Plan ==\nage: string, name: string\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#7L as string) AS age#45, cast(name#8 as string) AS name#46]\n +- Filter (age#7L > cast(21 as bigint))\n +- Relation[age#7L,name#8] json\n\n== Optimized Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#7L as string) AS age#45, name#8]\n +- Filter (isnotnull(age#7L) && (age#7L > 21))\n +- Relation[age#7L,name#8] json\n\n== Physical Plan ==\nCollectLimit 21\n+- *(1) Project [cast(age#7L as string) AS age#45, name#8]\n +- *(1) Filter (isnotnull(age#7L) && (age#7L > 21))\n +- *(1) FileScan json [age#7L,name#8] Batched: false, DataFilters: [isnotnull(age#7L), (age#7L > 21)], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [IsNotNull(age), GreaterThan(age,21)], ReadSchema: struct\n","sparkPlanInfo":{"nodeName":"CollectLimit","simpleString":"CollectLimit 21","children":[{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"Project","simpleString":"Project [cast(age#7L as string) AS age#45, name#8]","children":[{"nodeName":"Filter","simpleString":"Filter (isnotnull(age#7L) && (age#7L > 21))","children":[{"nodeName":"Scan json ","simpleString":"FileScan json [age#7L,name#8] Batched: false, DataFilters: [isnotnull(age#7L), (age#7L > 21)], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [IsNotNull(age), GreaterThan(age,21)], ReadSchema: struct","children":[],"metadata":{"Location":"InMemoryFileIndex[hdfs://node6:8020/user/spark/examples/src/main/resources/people.json]","ReadSchema":"struct","Format":"JSON","Batched":"false","PartitionFilters":"[]","PushedFilters":"[IsNotNull(age), GreaterThan(age,21)]","DataFilters":"[isnotnull(age#7L), (age#7L > 21)]"},"metrics":[{"name":"number of output rows","accumulatorId":162,"metricType":"sum"},{"name":"number of files","accumulatorId":163,"metricType":"sum"},{"name":"metadata time","accumulatorId":164,"metricType":"sum"},{"name":"scan time total (min, med, max)","accumulatorId":165,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":161,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration total (min, med, max)","accumulatorId":160,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":158,"metricType":"sum"},{"name":"shuffle write time total (min, med, max)","accumulatorId":159,"metricType":"nsTiming"},{"name":"records read","accumulatorId":156,"metricType":"sum"},{"name":"local bytes read total (min, med, max)","accumulatorId":154,"metricType":"size"},{"name":"fetch wait time total (min, med, max)","accumulatorId":155,"metricType":"timing"},{"name":"remote bytes read total (min, med, max)","accumulatorId":152,"metricType":"size"},{"name":"local blocks read","accumulatorId":151,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":150,"metricType":"sum"},{"name":"remote bytes read to disk total (min, med, max)","accumulatorId":153,"metricType":"size"},{"name":"shuffle bytes written total (min, med, max)","accumulatorId":157,"metricType":"size"}]},"time":1547788465156} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":3,"accumUpdates":[[163,1],[164,0]]} -{"Event":"SparkListenerJobStart","Job ID":4,"Submission Time":1547788465226,"Stage Infos":[{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:102","Number of Tasks":1,"RDD Info":[{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"37\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:102","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"32\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:102","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"FileScanRDD","Scope":"{\"id\":\"32\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:102","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"36\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:102","Parent IDs":[16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:102)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[4],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"3","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:102","Number of Tasks":1,"RDD Info":[{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"37\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:102","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"32\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:102","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"FileScanRDD","Scope":"{\"id\":\"32\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:102","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"36\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:102","Parent IDs":[16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:102)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788465227,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"3","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":0,"Attempt":0,"Launch Time":1547788465233,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":0,"Attempt":0,"Launch Time":1547788465233,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788465370,"Failed":false,"Killed":false,"Accumulables":[{"ID":160,"Name":"duration total (min, med, max)","Update":"56","Value":"55","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":161,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":162,"Name":"number of output rows","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":188,"Name":"internal.metrics.input.recordsRead","Update":3,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":187,"Name":"internal.metrics.input.bytesRead","Update":73,"Value":73,"Internal":true,"Count Failed Values":true},{"ID":170,"Name":"internal.metrics.resultSize","Update":1247,"Value":1247,"Internal":true,"Count Failed Values":true},{"ID":169,"Name":"internal.metrics.executorCpuTime","Update":83184993,"Value":83184993,"Internal":true,"Count Failed Values":true},{"ID":168,"Name":"internal.metrics.executorRunTime","Update":95,"Value":95,"Internal":true,"Count Failed Values":true},{"ID":167,"Name":"internal.metrics.executorDeserializeCpuTime","Update":18971260,"Value":18971260,"Internal":true,"Count Failed Values":true},{"ID":166,"Name":"internal.metrics.executorDeserializeTime","Update":28,"Value":28,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":28,"Executor Deserialize CPU Time":18971260,"Executor Run Time":95,"Executor CPU Time":83184993,"Result Size":1247,"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":73,"Records Read":3},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:102","Number of Tasks":1,"RDD Info":[{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"37\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:102","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"32\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:102","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"FileScanRDD","Scope":"{\"id\":\"32\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:102","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"36\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:102","Parent IDs":[16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:102)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788465227,"Completion Time":1547788465372,"Accumulables":[{"ID":167,"Name":"internal.metrics.executorDeserializeCpuTime","Value":18971260,"Internal":true,"Count Failed Values":true},{"ID":166,"Name":"internal.metrics.executorDeserializeTime","Value":28,"Internal":true,"Count Failed Values":true},{"ID":160,"Name":"duration total (min, med, max)","Value":"55","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":187,"Name":"internal.metrics.input.bytesRead","Value":73,"Internal":true,"Count Failed Values":true},{"ID":169,"Name":"internal.metrics.executorCpuTime","Value":83184993,"Internal":true,"Count Failed Values":true},{"ID":168,"Name":"internal.metrics.executorRunTime","Value":95,"Internal":true,"Count Failed Values":true},{"ID":162,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":161,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":188,"Name":"internal.metrics.input.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":170,"Name":"internal.metrics.resultSize","Value":1247,"Internal":true,"Count Failed Values":true}]}} -{"Event":"SparkListenerJobEnd","Job ID":4,"Completion Time":1547788465372,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":3,"time":1547788465374} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":4,"description":"show at SparkSQLExample.scala:110","details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#7L as string) AS age#62, cast(count#57L as string) AS count#63]\n +- Aggregate [age#7L], [age#7L, count(1) AS count#57L]\n +- Relation[age#7L,name#8] json\n\n== Analyzed Logical Plan ==\nage: string, count: string\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#7L as string) AS age#62, cast(count#57L as string) AS count#63]\n +- Aggregate [age#7L], [age#7L, count(1) AS count#57L]\n +- Relation[age#7L,name#8] json\n\n== Optimized Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Aggregate [age#7L], [cast(age#7L as string) AS age#62, cast(count(1) as string) AS count#63]\n +- Project [age#7L]\n +- Relation[age#7L,name#8] json\n\n== Physical Plan ==\nCollectLimit 21\n+- *(2) HashAggregate(keys=[age#7L], functions=[count(1)], output=[age#62, count#63])\n +- Exchange hashpartitioning(age#7L, 200)\n +- *(1) HashAggregate(keys=[age#7L], functions=[partial_count(1)], output=[age#7L, count#67L])\n +- *(1) FileScan json [age#7L] Batched: false, DataFilters: [], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n","sparkPlanInfo":{"nodeName":"CollectLimit","simpleString":"CollectLimit 21","children":[{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[age#7L], functions=[count(1)])","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Exchange","simpleString":"Exchange hashpartitioning(age#7L, 200)","children":[{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"HashAggregate","simpleString":"HashAggregate(keys=[age#7L], functions=[partial_count(1)])","children":[{"nodeName":"Scan json ","simpleString":"FileScan json [age#7L] Batched: false, DataFilters: [], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct","children":[],"metadata":{"Location":"InMemoryFileIndex[hdfs://node6:8020/user/spark/examples/src/main/resources/people.json]","ReadSchema":"struct","Format":"JSON","Batched":"false","PartitionFilters":"[]","PushedFilters":"[]","DataFilters":"[]"},"metrics":[{"name":"number of output rows","accumulatorId":235,"metricType":"sum"},{"name":"number of files","accumulatorId":236,"metricType":"sum"},{"name":"metadata time","accumulatorId":237,"metricType":"sum"},{"name":"scan time total (min, med, max)","accumulatorId":238,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"spill size total (min, med, max)","accumulatorId":232,"metricType":"size"},{"name":"aggregate time total (min, med, max)","accumulatorId":233,"metricType":"timing"},{"name":"peak memory total (min, med, max)","accumulatorId":231,"metricType":"size"},{"name":"number of output rows","accumulatorId":230,"metricType":"sum"},{"name":"avg hash probe (min, med, max)","accumulatorId":234,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration total (min, med, max)","accumulatorId":229,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":211,"metricType":"sum"},{"name":"shuffle write time total (min, med, max)","accumulatorId":212,"metricType":"nsTiming"},{"name":"records read","accumulatorId":209,"metricType":"sum"},{"name":"local bytes read total (min, med, max)","accumulatorId":207,"metricType":"size"},{"name":"fetch wait time total (min, med, max)","accumulatorId":208,"metricType":"timing"},{"name":"remote bytes read total (min, med, max)","accumulatorId":205,"metricType":"size"},{"name":"local blocks read","accumulatorId":204,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":203,"metricType":"sum"},{"name":"data size total (min, med, max)","accumulatorId":202,"metricType":"size"},{"name":"remote bytes read to disk total (min, med, max)","accumulatorId":206,"metricType":"size"},{"name":"shuffle bytes written total (min, med, max)","accumulatorId":210,"metricType":"size"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"spill size total (min, med, max)","accumulatorId":226,"metricType":"size"},{"name":"aggregate time total (min, med, max)","accumulatorId":227,"metricType":"timing"},{"name":"peak memory total (min, med, max)","accumulatorId":225,"metricType":"size"},{"name":"number of output rows","accumulatorId":224,"metricType":"sum"},{"name":"avg hash probe (min, med, max)","accumulatorId":228,"metricType":"average"}]}],"metadata":{},"metrics":[{"name":"duration total (min, med, max)","accumulatorId":223,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":221,"metricType":"sum"},{"name":"shuffle write time total (min, med, max)","accumulatorId":222,"metricType":"nsTiming"},{"name":"records read","accumulatorId":219,"metricType":"sum"},{"name":"local bytes read total (min, med, max)","accumulatorId":217,"metricType":"size"},{"name":"fetch wait time total (min, med, max)","accumulatorId":218,"metricType":"timing"},{"name":"remote bytes read total (min, med, max)","accumulatorId":215,"metricType":"size"},{"name":"local blocks read","accumulatorId":214,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":213,"metricType":"sum"},{"name":"remote bytes read to disk total (min, med, max)","accumulatorId":216,"metricType":"size"},{"name":"shuffle bytes written total (min, med, max)","accumulatorId":220,"metricType":"size"}]},"time":1547788465472} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":4,"accumUpdates":[[236,1],[237,0]]} -{"Event":"SparkListenerJobStart","Job ID":5,"Submission Time":1547788465753,"Stage Infos":[{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":1,"RDD Info":[{"RDD ID":21,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":19,"Name":"FileScanRDD","Scope":"{\"id\":\"46\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"46\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]},{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":1,"RDD Info":[{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"50\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[5],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[5,6],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"4","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":1,"RDD Info":[{"RDD ID":21,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":19,"Name":"FileScanRDD","Scope":"{\"id\":\"46\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"46\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788465757,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"4","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":0,"Attempt":0,"Launch Time":1547788465770,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":0,"Attempt":0,"Launch Time":1547788465770,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788466838,"Failed":false,"Killed":false,"Accumulables":[{"ID":212,"Name":"shuffle write time total (min, med, max)","Update":"59882301","Value":"59882300","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":211,"Name":"shuffle records written","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":210,"Name":"shuffle bytes written total (min, med, max)","Update":"185","Value":"184","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":202,"Name":"data size total (min, med, max)","Update":"71","Value":"70","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":229,"Name":"duration total (min, med, max)","Update":"633","Value":"632","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":234,"Name":"avg hash probe (min, med, max)","Update":"10","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":230,"Name":"number of output rows","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":231,"Name":"peak memory total (min, med, max)","Update":"17039359","Value":"17039358","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":233,"Name":"aggregate time total (min, med, max)","Update":"170","Value":"169","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":235,"Name":"number of output rows","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":261,"Name":"internal.metrics.input.recordsRead","Update":3,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":260,"Name":"internal.metrics.input.bytesRead","Update":73,"Value":73,"Internal":true,"Count Failed Values":true},{"ID":259,"Name":"internal.metrics.shuffle.write.writeTime","Update":59882302,"Value":59882302,"Internal":true,"Count Failed Values":true},{"ID":258,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":3,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":257,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":186,"Value":186,"Internal":true,"Count Failed Values":true},{"ID":248,"Name":"internal.metrics.peakExecutionMemory","Update":17039360,"Value":17039360,"Internal":true,"Count Failed Values":true},{"ID":245,"Name":"internal.metrics.resultSerializationTime","Update":3,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":243,"Name":"internal.metrics.resultSize","Update":2687,"Value":2687,"Internal":true,"Count Failed Values":true},{"ID":242,"Name":"internal.metrics.executorCpuTime","Update":710856733,"Value":710856733,"Internal":true,"Count Failed Values":true},{"ID":241,"Name":"internal.metrics.executorRunTime","Update":871,"Value":871,"Internal":true,"Count Failed Values":true},{"ID":240,"Name":"internal.metrics.executorDeserializeCpuTime","Update":162794413,"Value":162794413,"Internal":true,"Count Failed Values":true},{"ID":239,"Name":"internal.metrics.executorDeserializeTime","Update":177,"Value":177,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":177,"Executor Deserialize CPU Time":162794413,"Executor Run Time":871,"Executor CPU Time":710856733,"Result Size":2687,"JVM GC Time":0,"Result Serialization Time":3,"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":186,"Shuffle Write Time":59882302,"Shuffle Records Written":3},"Input Metrics":{"Bytes Read":73,"Records Read":3},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":1,"RDD Info":[{"RDD ID":21,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":19,"Name":"FileScanRDD","Scope":"{\"id\":\"46\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"46\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788465757,"Completion Time":1547788466840,"Accumulables":[{"ID":245,"Name":"internal.metrics.resultSerializationTime","Value":3,"Internal":true,"Count Failed Values":true},{"ID":235,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":211,"Name":"shuffle records written","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":202,"Name":"data size total (min, med, max)","Value":"70","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":229,"Name":"duration total (min, med, max)","Value":"632","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":259,"Name":"internal.metrics.shuffle.write.writeTime","Value":59882302,"Internal":true,"Count Failed Values":true},{"ID":241,"Name":"internal.metrics.executorRunTime","Value":871,"Internal":true,"Count Failed Values":true},{"ID":240,"Name":"internal.metrics.executorDeserializeCpuTime","Value":162794413,"Internal":true,"Count Failed Values":true},{"ID":258,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":3,"Internal":true,"Count Failed Values":true},{"ID":234,"Name":"avg hash probe (min, med, max)","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":243,"Name":"internal.metrics.resultSize","Value":2687,"Internal":true,"Count Failed Values":true},{"ID":261,"Name":"internal.metrics.input.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":210,"Name":"shuffle bytes written total (min, med, max)","Value":"184","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":231,"Name":"peak memory total (min, med, max)","Value":"17039358","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":230,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":257,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":186,"Internal":true,"Count Failed Values":true},{"ID":212,"Name":"shuffle write time total (min, med, max)","Value":"59882300","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":239,"Name":"internal.metrics.executorDeserializeTime","Value":177,"Internal":true,"Count Failed Values":true},{"ID":248,"Name":"internal.metrics.peakExecutionMemory","Value":17039360,"Internal":true,"Count Failed Values":true},{"ID":260,"Name":"internal.metrics.input.bytesRead","Value":73,"Internal":true,"Count Failed Values":true},{"ID":233,"Name":"aggregate time total (min, med, max)","Value":"169","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":242,"Name":"internal.metrics.executorCpuTime","Value":710856733,"Internal":true,"Count Failed Values":true}]}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":1,"RDD Info":[{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"50\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[5],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788466853,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"4","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":0,"Attempt":0,"Launch Time":1547788466859,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":0,"Attempt":0,"Launch Time":1547788466859,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788467141,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"16","Value":"15","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"262142","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":227,"Name":"aggregate time total (min, med, max)","Update":"1","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":281,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":280,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":279,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":278,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":277,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":276,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":275,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":273,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":262144,"Internal":true,"Count Failed Values":true},{"ID":268,"Name":"internal.metrics.resultSize","Update":3617,"Value":3617,"Internal":true,"Count Failed Values":true},{"ID":267,"Name":"internal.metrics.executorCpuTime","Update":172136588,"Value":172136588,"Internal":true,"Count Failed Values":true},{"ID":266,"Name":"internal.metrics.executorRunTime","Update":192,"Value":192,"Internal":true,"Count Failed Values":true},{"ID":265,"Name":"internal.metrics.executorDeserializeCpuTime","Update":65342261,"Value":65342261,"Internal":true,"Count Failed Values":true},{"ID":264,"Name":"internal.metrics.executorDeserializeTime","Update":76,"Value":76,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":76,"Executor Deserialize CPU Time":65342261,"Executor Run Time":192,"Executor CPU Time":172136588,"Result Size":3617,"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":"SparkListenerStageCompleted","Stage Info":{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":1,"RDD Info":[{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"50\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[5],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788466853,"Completion Time":1547788467143,"Accumulables":[{"ID":227,"Name":"aggregate time total (min, med, max)","Value":"0","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":280,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":265,"Name":"internal.metrics.executorDeserializeCpuTime","Value":65342261,"Internal":true,"Count Failed Values":true},{"ID":223,"Name":"duration total (min, med, max)","Value":"15","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":277,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":268,"Name":"internal.metrics.resultSize","Value":3617,"Internal":true,"Count Failed Values":true},{"ID":267,"Name":"internal.metrics.executorCpuTime","Value":172136588,"Internal":true,"Count Failed Values":true},{"ID":276,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":225,"Name":"peak memory total (min, med, max)","Value":"262142","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":279,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":264,"Name":"internal.metrics.executorDeserializeTime","Value":76,"Internal":true,"Count Failed Values":true},{"ID":273,"Name":"internal.metrics.peakExecutionMemory","Value":262144,"Internal":true,"Count Failed Values":true},{"ID":281,"Name":"internal.metrics.shuffle.read.recordsRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":266,"Name":"internal.metrics.executorRunTime","Value":192,"Internal":true,"Count Failed Values":true},{"ID":275,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":278,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true}]}} -{"Event":"SparkListenerJobEnd","Job ID":5,"Completion Time":1547788467143,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerJobStart","Job ID":6,"Submission Time":1547788467155,"Stage Infos":[{"Stage ID":7,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":1,"RDD Info":[{"RDD ID":21,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":19,"Name":"FileScanRDD","Scope":"{\"id\":\"46\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"46\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]},{"Stage ID":8,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":4,"RDD Info":[{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"50\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[7],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[7,8],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"4","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":8,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":4,"RDD Info":[{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"50\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[7],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788467158,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"4","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerTaskStart","Stage ID":8,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":3,"Attempt":0,"Launch Time":1547788467166,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":8,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1547788467167,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":8,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1547788467255,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":8,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":3,"Attempt":0,"Launch Time":1547788467166,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788467256,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"29","Value":"44","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":228,"Name":"avg hash probe (min, med, max)","Update":"10","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":224,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"17039359","Value":"17301501","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":227,"Name":"aggregate time total (min, med, max)","Update":"19","Value":"19","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":204,"Name":"local blocks read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":207,"Name":"local bytes read total (min, med, max)","Update":"62","Value":"60","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":209,"Name":"records read","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":306,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":305,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":304,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":63,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":303,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":302,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":301,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":300,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":298,"Name":"internal.metrics.peakExecutionMemory","Update":17039360,"Value":17039360,"Internal":true,"Count Failed Values":true},{"ID":293,"Name":"internal.metrics.resultSize","Update":3656,"Value":3656,"Internal":true,"Count Failed Values":true},{"ID":292,"Name":"internal.metrics.executorCpuTime","Update":50963610,"Value":50963610,"Internal":true,"Count Failed Values":true},{"ID":291,"Name":"internal.metrics.executorRunTime","Update":51,"Value":51,"Internal":true,"Count Failed Values":true},{"ID":290,"Name":"internal.metrics.executorDeserializeCpuTime","Update":18570925,"Value":18570925,"Internal":true,"Count Failed Values":true},{"ID":289,"Name":"internal.metrics.executorDeserializeTime","Update":25,"Value":25,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":25,"Executor Deserialize CPU Time":18570925,"Executor Run Time":51,"Executor CPU Time":50963610,"Result Size":3656,"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":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":63,"Total Records Read":1},"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":8,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1547788467301,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":8,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1547788467255,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788467302,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"14","Value":"58","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"17563644","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":306,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":305,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":304,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":303,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":302,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":301,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":300,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":298,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":17301504,"Internal":true,"Count Failed Values":true},{"ID":293,"Name":"internal.metrics.resultSize","Update":3617,"Value":7273,"Internal":true,"Count Failed Values":true},{"ID":292,"Name":"internal.metrics.executorCpuTime","Update":20052870,"Value":71016480,"Internal":true,"Count Failed Values":true},{"ID":291,"Name":"internal.metrics.executorRunTime","Update":20,"Value":71,"Internal":true,"Count Failed Values":true},{"ID":290,"Name":"internal.metrics.executorDeserializeCpuTime","Update":13903504,"Value":32474429,"Internal":true,"Count Failed Values":true},{"ID":289,"Name":"internal.metrics.executorDeserializeTime","Update":14,"Value":39,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":14,"Executor Deserialize CPU Time":13903504,"Executor Run Time":20,"Executor CPU Time":20052870,"Result Size":3617,"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":8,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1547788467301,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788467346,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"17","Value":"75","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"17825787","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":306,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":305,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":304,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":303,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":302,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":301,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":300,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":298,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":17563648,"Internal":true,"Count Failed Values":true},{"ID":293,"Name":"internal.metrics.resultSize","Update":3617,"Value":10890,"Internal":true,"Count Failed Values":true},{"ID":292,"Name":"internal.metrics.executorCpuTime","Update":17420764,"Value":88437244,"Internal":true,"Count Failed Values":true},{"ID":291,"Name":"internal.metrics.executorRunTime","Update":22,"Value":93,"Internal":true,"Count Failed Values":true},{"ID":290,"Name":"internal.metrics.executorDeserializeCpuTime","Update":11738163,"Value":44212592,"Internal":true,"Count Failed Values":true},{"ID":289,"Name":"internal.metrics.executorDeserializeTime","Update":13,"Value":52,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":13,"Executor Deserialize CPU Time":11738163,"Executor Run Time":22,"Executor CPU Time":17420764,"Result Size":3617,"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":8,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1547788467167,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788470544,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"209","Value":"284","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"18087930","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":227,"Name":"aggregate time total (min, med, max)","Update":"8","Value":"25","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":306,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":305,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":304,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":303,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":302,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":301,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":300,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":298,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":17825792,"Internal":true,"Count Failed Values":true},{"ID":295,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":294,"Name":"internal.metrics.jvmGCTime","Update":162,"Value":162,"Internal":true,"Count Failed Values":true},{"ID":293,"Name":"internal.metrics.resultSize","Update":3703,"Value":14593,"Internal":true,"Count Failed Values":true},{"ID":292,"Name":"internal.metrics.executorCpuTime","Update":1088992840,"Value":1177430084,"Internal":true,"Count Failed Values":true},{"ID":291,"Name":"internal.metrics.executorRunTime","Update":1330,"Value":1423,"Internal":true,"Count Failed Values":true},{"ID":290,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1446326008,"Value":1490538600,"Internal":true,"Count Failed Values":true},{"ID":289,"Name":"internal.metrics.executorDeserializeTime","Update":1996,"Value":2048,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1996,"Executor Deserialize CPU Time":1446326008,"Executor Run Time":1330,"Executor CPU Time":1088992840,"Result Size":3703,"JVM GC Time":162,"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":"SparkListenerStageCompleted","Stage Info":{"Stage ID":8,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":4,"RDD Info":[{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"50\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[7],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788467158,"Completion Time":1547788470547,"Accumulables":[{"ID":227,"Name":"aggregate time total (min, med, max)","Value":"25","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":209,"Name":"records read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":304,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":63,"Internal":true,"Count Failed Values":true},{"ID":295,"Name":"internal.metrics.resultSerializationTime","Value":1,"Internal":true,"Count Failed Values":true},{"ID":298,"Name":"internal.metrics.peakExecutionMemory","Value":17825792,"Internal":true,"Count Failed Values":true},{"ID":289,"Name":"internal.metrics.executorDeserializeTime","Value":2048,"Internal":true,"Count Failed Values":true},{"ID":301,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":292,"Name":"internal.metrics.executorCpuTime","Value":1177430084,"Internal":true,"Count Failed Values":true},{"ID":223,"Name":"duration total (min, med, max)","Value":"284","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":300,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":294,"Name":"internal.metrics.jvmGCTime","Value":162,"Internal":true,"Count Failed Values":true},{"ID":303,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":225,"Name":"peak memory total (min, med, max)","Value":"18087930","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":306,"Name":"internal.metrics.shuffle.read.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":207,"Name":"local bytes read total (min, med, max)","Value":"60","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":228,"Name":"avg hash probe (min, med, max)","Value":"10","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":291,"Name":"internal.metrics.executorRunTime","Value":1423,"Internal":true,"Count Failed Values":true},{"ID":204,"Name":"local blocks read","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":290,"Name":"internal.metrics.executorDeserializeCpuTime","Value":1490538600,"Internal":true,"Count Failed Values":true},{"ID":293,"Name":"internal.metrics.resultSize","Value":14593,"Internal":true,"Count Failed Values":true},{"ID":302,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":224,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":305,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true}]}} -{"Event":"SparkListenerJobEnd","Job ID":6,"Completion Time":1547788470548,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerJobStart","Job ID":7,"Submission Time":1547788470558,"Stage Infos":[{"Stage ID":9,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":1,"RDD Info":[{"RDD ID":21,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":19,"Name":"FileScanRDD","Scope":"{\"id\":\"46\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"46\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]},{"Stage ID":10,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":20,"RDD Info":[{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"50\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[9],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[9,10],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"4","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":10,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":20,"RDD Info":[{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"50\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[9],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788470562,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"4","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerTaskStart","Stage ID":10,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":0,"Attempt":0,"Launch Time":1547788470568,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":10,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":1,"Attempt":0,"Launch Time":1547788470569,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":10,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":2,"Attempt":0,"Launch Time":1547788470626,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":0,"Attempt":0,"Launch Time":1547788470568,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788470627,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"13","Value":"297","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"18350073","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":262144,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3617,"Value":3617,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":18437250,"Value":18437250,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":18,"Value":18,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":20743656,"Value":20743656,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":29,"Value":29,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":29,"Executor Deserialize CPU Time":20743656,"Executor Run Time":18,"Executor CPU Time":18437250,"Result Size":3617,"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":10,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":3,"Attempt":0,"Launch Time":1547788470668,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":2,"Attempt":0,"Launch Time":1547788470626,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788470668,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"14","Value":"311","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"18612216","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":524288,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3617,"Value":7234,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":19340397,"Value":37777647,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":18,"Value":36,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":12502299,"Value":33245955,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":13,"Value":42,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":13,"Executor Deserialize CPU Time":12502299,"Executor Run Time":18,"Executor CPU Time":19340397,"Result Size":3617,"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":10,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":4,"Attempt":0,"Launch Time":1547788470699,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":3,"Attempt":0,"Launch Time":1547788470668,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788470699,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"9","Value":"320","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"18874359","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":786432,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3617,"Value":10851,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":13392141,"Value":51169788,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":14,"Value":50,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8423156,"Value":41669111,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":8,"Value":50,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":8,"Executor Deserialize CPU Time":8423156,"Executor Run Time":14,"Executor CPU Time":13392141,"Result Size":3617,"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":10,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":5,"Attempt":0,"Launch Time":1547788470732,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":4,"Attempt":0,"Launch Time":1547788470699,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788470733,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"9","Value":"329","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"19136502","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":1048576,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3617,"Value":14468,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":13950272,"Value":65120060,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":14,"Value":64,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":10176458,"Value":51845569,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":10,"Value":60,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":10,"Executor Deserialize CPU Time":10176458,"Executor Run Time":14,"Executor CPU Time":13950272,"Result Size":3617,"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":10,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":6,"Attempt":0,"Launch Time":1547788470767,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":5,"Attempt":0,"Launch Time":1547788470732,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788470767,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"10","Value":"339","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"19398645","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":1310720,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3617,"Value":18085,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":15018675,"Value":80138735,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":15,"Value":79,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":10862497,"Value":62708066,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":11,"Value":71,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":11,"Executor Deserialize CPU Time":10862497,"Executor Run Time":15,"Executor CPU Time":15018675,"Result Size":3617,"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":10,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":7,"Attempt":0,"Launch Time":1547788470799,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":6,"Attempt":0,"Launch Time":1547788470767,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788470799,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"7","Value":"346","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"19660788","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":1572864,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3617,"Value":21702,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":12762090,"Value":92900825,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":13,"Value":92,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":10447486,"Value":73155552,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":10,"Value":81,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":10,"Executor Deserialize CPU Time":10447486,"Executor Run Time":13,"Executor CPU Time":12762090,"Result Size":3617,"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":10,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":8,"Attempt":0,"Launch Time":1547788470824,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":7,"Attempt":0,"Launch Time":1547788470799,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788470825,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"352","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"19922931","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":1835008,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3617,"Value":25319,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":9882914,"Value":102783739,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":10,"Value":102,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7633805,"Value":80789357,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":88,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":7633805,"Executor Run Time":10,"Executor CPU Time":9882914,"Result Size":3617,"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":10,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":9,"Attempt":0,"Launch Time":1547788470846,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":8,"Attempt":0,"Launch Time":1547788470824,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788470847,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"357","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"20185074","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":2097152,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3617,"Value":28936,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":9266539,"Value":112050278,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":9,"Value":111,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6207804,"Value":86997161,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":94,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":6207804,"Executor Run Time":9,"Executor CPU Time":9266539,"Result Size":3617,"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":10,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":10,"Attempt":0,"Launch Time":1547788470866,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":1,"Attempt":0,"Launch Time":1547788470569,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788470867,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"34","Value":"391","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"20447217","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":227,"Name":"aggregate time total (min, med, max)","Update":"1","Value":"18","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":2359296,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3617,"Value":32553,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":34825296,"Value":146875574,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":45,"Value":156,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":35461794,"Value":122458955,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":236,"Value":330,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":236,"Executor Deserialize CPU Time":35461794,"Executor Run Time":45,"Executor CPU Time":34825296,"Result Size":3617,"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":10,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":11,"Attempt":0,"Launch Time":1547788470869,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":9,"Attempt":0,"Launch Time":1547788470846,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788470870,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"7","Value":"398","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"20709360","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":2621440,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3617,"Value":36170,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":9989200,"Value":156864774,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":10,"Value":166,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5843063,"Value":128302018,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":336,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5843063,"Executor Run Time":10,"Executor CPU Time":9989200,"Result Size":3617,"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":10,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":12,"Attempt":0,"Launch Time":1547788470891,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":11,"Attempt":0,"Launch Time":1547788470869,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788470892,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"403","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"20971503","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":2883584,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3617,"Value":39787,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":8835589,"Value":165700363,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":9,"Value":175,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5785895,"Value":134087913,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":341,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5785895,"Executor Run Time":9,"Executor CPU Time":8835589,"Result Size":3617,"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":10,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":13,"Attempt":0,"Launch Time":1547788470917,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":12,"Attempt":0,"Launch Time":1547788470891,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788470918,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"7","Value":"410","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"21233646","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":3145728,"Internal":true,"Count Failed Values":true},{"ID":320,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3660,"Value":43447,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":9468324,"Value":175168687,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":10,"Value":185,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7086416,"Value":141174329,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":348,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":7086416,"Executor Run Time":10,"Executor CPU Time":9468324,"Result Size":3660,"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":10,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":14,"Attempt":0,"Launch Time":1547788470930,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":10,"Attempt":0,"Launch Time":1547788470866,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788470930,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"27","Value":"437","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"21495789","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":3407872,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3617,"Value":47064,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":23223894,"Value":198392581,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":33,"Value":218,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":12667423,"Value":153841752,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":20,"Value":368,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":20,"Executor Deserialize CPU Time":12667423,"Executor Run Time":33,"Executor CPU Time":23223894,"Result Size":3617,"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":10,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":15,"Attempt":0,"Launch Time":1547788470939,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":13,"Attempt":0,"Launch Time":1547788470917,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788470940,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"442","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"21757932","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":3670016,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3617,"Value":50681,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":9181493,"Value":207574074,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":9,"Value":227,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6095760,"Value":159937512,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":374,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":6095760,"Executor Run Time":9,"Executor CPU Time":9181493,"Result Size":3617,"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":10,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":16,"Attempt":0,"Launch Time":1547788470961,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":15,"Attempt":0,"Launch Time":1547788470939,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788470962,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"448","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"22020075","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":3932160,"Internal":true,"Count Failed Values":true},{"ID":320,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3660,"Value":54341,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":9000681,"Value":216574755,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":9,"Value":236,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5739878,"Value":165677390,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":380,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5739878,"Executor Run Time":9,"Executor CPU Time":9000681,"Result Size":3660,"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":10,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":17,"Attempt":0,"Launch Time":1547788470983,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":16,"Attempt":0,"Launch Time":1547788470961,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788470984,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"454","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"22282218","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":4194304,"Internal":true,"Count Failed Values":true},{"ID":320,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3660,"Value":58001,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":9278513,"Value":225853268,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":9,"Value":245,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5693904,"Value":171371294,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":386,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5693904,"Executor Run Time":9,"Executor CPU Time":9278513,"Result Size":3660,"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":10,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":18,"Attempt":0,"Launch Time":1547788471004,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":17,"Attempt":0,"Launch Time":1547788470983,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471005,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"459","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"22544361","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":4456448,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3617,"Value":61618,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":8992812,"Value":234846080,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":9,"Value":254,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5709977,"Value":177081271,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":392,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5709977,"Executor Run Time":9,"Executor CPU Time":8992812,"Result Size":3617,"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":10,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":19,"Attempt":0,"Launch Time":1547788471006,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":14,"Attempt":0,"Launch Time":1547788470930,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471006,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"42","Value":"501","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"22806504","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":227,"Name":"aggregate time total (min, med, max)","Update":"15","Value":"25","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":4718592,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3617,"Value":65235,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":20343990,"Value":255190070,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":51,"Value":305,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":9788084,"Value":186869355,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":11,"Value":403,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":11,"Executor Deserialize CPU Time":9788084,"Executor Run Time":51,"Executor CPU Time":20343990,"Result Size":3617,"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":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":18,"Attempt":0,"Launch Time":1547788471004,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471026,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"506","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"23068647","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":4980736,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3617,"Value":68852,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":8887889,"Value":264077959,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":9,"Value":314,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5985079,"Value":192854434,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":409,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5985079,"Executor Run Time":9,"Executor CPU Time":8887889,"Result Size":3617,"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":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":19,"Attempt":0,"Launch Time":1547788471006,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471052,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"16","Value":"522","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"23330790","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.resultSize","Update":3617,"Value":72469,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Update":20622106,"Value":284700065,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Update":22,"Value":336,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Update":11957927,"Value":204812361,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Update":14,"Value":423,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":14,"Executor Deserialize CPU Time":11957927,"Executor Run Time":22,"Executor CPU Time":20622106,"Result Size":3617,"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":"SparkListenerStageCompleted","Stage Info":{"Stage ID":10,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":20,"RDD Info":[{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"50\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[9],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788470562,"Completion Time":1547788471053,"Accumulables":[{"ID":227,"Name":"aggregate time total (min, med, max)","Value":"25","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":331,"Name":"internal.metrics.shuffle.read.recordsRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.executorRunTime","Value":336,"Internal":true,"Count Failed Values":true},{"ID":325,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":223,"Name":"duration total (min, med, max)","Value":"522","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":318,"Name":"internal.metrics.resultSize","Value":72469,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":225,"Name":"peak memory total (min, med, max)","Value":"23330790","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":315,"Name":"internal.metrics.executorDeserializeCpuTime","Value":204812361,"Internal":true,"Count Failed Values":true},{"ID":323,"Name":"internal.metrics.peakExecutionMemory","Value":5242880,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.executorCpuTime","Value":284700065,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":320,"Name":"internal.metrics.resultSerializationTime","Value":3,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.executorDeserializeTime","Value":423,"Internal":true,"Count Failed Values":true}]}} -{"Event":"SparkListenerJobEnd","Job ID":7,"Completion Time":1547788471054,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerJobStart","Job ID":8,"Submission Time":1547788471064,"Stage Infos":[{"Stage ID":12,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":100,"RDD Info":[{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"50\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[11],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]},{"Stage ID":11,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":1,"RDD Info":[{"RDD ID":21,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":19,"Name":"FileScanRDD","Scope":"{\"id\":\"46\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"46\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[12,11],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"4","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":12,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":100,"RDD Info":[{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"50\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[11],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788471074,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"4","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":17,"Attempt":0,"Launch Time":1547788471082,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":32,"Index":0,"Attempt":0,"Launch Time":1547788471083,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":33,"Index":1,"Attempt":0,"Launch Time":1547788471120,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":31,"Index":17,"Attempt":0,"Launch Time":1547788471082,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471121,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"528","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":228,"Name":"avg hash probe (min, med, max)","Update":"10","Value":"20","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":224,"Name":"number of output rows","Update":"1","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"17039359","Value":"40370149","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":227,"Name":"aggregate time total (min, med, max)","Update":"0","Value":"23","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":204,"Name":"local blocks read","Update":"1","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":207,"Name":"local bytes read total (min, med, max)","Update":"59","Value":"96","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":209,"Name":"records read","Update":"1","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":60,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":17039360,"Value":17039360,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3643,"Value":3643,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":10444642,"Value":10444642,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":11,"Value":11,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":11614540,"Value":11614540,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":20,"Value":20,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":20,"Executor Deserialize CPU Time":11614540,"Executor Run Time":11,"Executor CPU Time":10444642,"Result Size":3643,"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":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":60,"Total Records Read":1},"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":34,"Index":2,"Attempt":0,"Launch Time":1547788471147,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":33,"Index":1,"Attempt":0,"Launch Time":1547788471120,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471147,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"7","Value":"535","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"40632292","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":17301504,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":7260,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":11458391,"Value":21903033,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":12,"Value":23,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7200707,"Value":18815247,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":27,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":7200707,"Executor Run Time":12,"Executor CPU Time":11458391,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":35,"Index":3,"Attempt":0,"Launch Time":1547788471158,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":32,"Index":0,"Attempt":0,"Launch Time":1547788471083,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471159,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"23","Value":"558","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"40894435","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":17563648,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":10877,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":18955449,"Value":40858482,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":28,"Value":51,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":22943627,"Value":41758874,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":36,"Value":63,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":36,"Executor Deserialize CPU Time":22943627,"Executor Run Time":28,"Executor CPU Time":18955449,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":36,"Index":4,"Attempt":0,"Launch Time":1547788471172,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":34,"Index":2,"Attempt":0,"Launch Time":1547788471147,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471172,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"564","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"41156578","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":17825792,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":14494,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9948802,"Value":50807284,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":11,"Value":62,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6934447,"Value":48693321,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":70,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":6934447,"Executor Run Time":11,"Executor CPU Time":9948802,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":37,"Index":5,"Attempt":0,"Launch Time":1547788471191,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":35,"Index":3,"Attempt":0,"Launch Time":1547788471158,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471192,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"10","Value":"574","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"41418721","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":18087936,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":18111,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":15075464,"Value":65882748,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":15,"Value":77,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7974927,"Value":56668248,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":8,"Value":78,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":8,"Executor Deserialize CPU Time":7974927,"Executor Run Time":15,"Executor CPU Time":15075464,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":38,"Index":6,"Attempt":0,"Launch Time":1547788471192,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":36,"Index":4,"Attempt":0,"Launch Time":1547788471172,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471193,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"580","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"41680864","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":18350080,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":21728,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9061776,"Value":74944524,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":86,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5539711,"Value":62207959,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":84,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5539711,"Executor Run Time":9,"Executor CPU Time":9061776,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":39,"Index":7,"Attempt":0,"Launch Time":1547788471224,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":38,"Index":6,"Attempt":0,"Launch Time":1547788471192,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471225,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"10","Value":"590","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"41943007","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":18612224,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":25345,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":12191130,"Value":87135654,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":15,"Value":101,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7951278,"Value":70159237,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":8,"Value":92,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":8,"Executor Deserialize CPU Time":7951278,"Executor Run Time":15,"Executor CPU Time":12191130,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":40,"Index":8,"Attempt":0,"Launch Time":1547788471238,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":37,"Index":5,"Attempt":0,"Launch Time":1547788471191,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471239,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"13","Value":"603","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"42205150","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":18874368,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":28962,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":14913138,"Value":102048792,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":18,"Value":119,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":10083135,"Value":80242372,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":11,"Value":103,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":11,"Executor Deserialize CPU Time":10083135,"Executor Run Time":18,"Executor CPU Time":14913138,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":41,"Index":9,"Attempt":0,"Launch Time":1547788471251,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":39,"Index":7,"Attempt":0,"Launch Time":1547788471224,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471251,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"7","Value":"610","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"42467293","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":19136512,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":32579,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":11277425,"Value":113326217,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":12,"Value":131,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7366758,"Value":87609130,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":110,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":7366758,"Executor Run Time":12,"Executor CPU Time":11277425,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":42,"Index":10,"Attempt":0,"Launch Time":1547788471272,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":40,"Index":8,"Attempt":0,"Launch Time":1547788471238,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471273,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"10","Value":"620","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"42729436","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":19398656,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":36196,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":13862819,"Value":127189036,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":15,"Value":146,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8801575,"Value":96410705,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":10,"Value":120,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":10,"Executor Deserialize CPU Time":8801575,"Executor Run Time":15,"Executor CPU Time":13862819,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":43,"Index":11,"Attempt":0,"Launch Time":1547788471274,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":41,"Index":9,"Attempt":0,"Launch Time":1547788471251,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471274,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"626","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"42991579","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":19660800,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":39813,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9404567,"Value":136593603,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":10,"Value":156,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5976060,"Value":102386765,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":126,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5976060,"Executor Run Time":10,"Executor CPU Time":9404567,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":44,"Index":12,"Attempt":0,"Launch Time":1547788471298,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":43,"Index":11,"Attempt":0,"Launch Time":1547788471274,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471299,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"631","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"43253722","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":19922944,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":43430,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9384454,"Value":145978057,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":165,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8651421,"Value":111038186,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":9,"Value":135,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":9,"Executor Deserialize CPU Time":8651421,"Executor Run Time":9,"Executor CPU Time":9384454,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":45,"Index":13,"Attempt":0,"Launch Time":1547788471308,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":42,"Index":10,"Attempt":0,"Launch Time":1547788471272,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471308,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"11","Value":"642","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"43515865","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":20185088,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":47047,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":14241163,"Value":160219220,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":16,"Value":181,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8736676,"Value":119774862,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":10,"Value":145,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":10,"Executor Deserialize CPU Time":8736676,"Executor Run Time":16,"Executor CPU Time":14241163,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":46,"Index":14,"Attempt":0,"Launch Time":1547788471320,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":44,"Index":12,"Attempt":0,"Launch Time":1547788471298,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471320,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"647","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"43778008","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":20447232,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":50664,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":8133337,"Value":168352557,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":190,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5486744,"Value":125261606,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":151,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5486744,"Executor Run Time":9,"Executor CPU Time":8133337,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":47,"Index":15,"Attempt":0,"Launch Time":1547788471342,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":46,"Index":14,"Attempt":0,"Launch Time":1547788471320,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471342,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"653","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"44040151","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":20709376,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":54281,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9024908,"Value":177377465,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":10,"Value":200,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4887146,"Value":130148752,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":156,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4887146,"Executor Run Time":10,"Executor CPU Time":9024908,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":48,"Index":16,"Attempt":0,"Launch Time":1547788471348,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":45,"Index":13,"Attempt":0,"Launch Time":1547788471308,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471348,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"12","Value":"665","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"44302294","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":227,"Name":"aggregate time total (min, med, max)","Update":"0","Value":"9","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":20971520,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":57898,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":14226807,"Value":191604272,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":17,"Value":217,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8717459,"Value":138866211,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":12,"Value":168,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":12,"Executor Deserialize CPU Time":8717459,"Executor Run Time":17,"Executor CPU Time":14226807,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":49,"Index":18,"Attempt":0,"Launch Time":1547788471364,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":47,"Index":15,"Attempt":0,"Launch Time":1547788471342,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471364,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"671","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"44564437","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":21233664,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":61515,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9689284,"Value":201293556,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":10,"Value":227,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4958749,"Value":143824960,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":173,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4958749,"Executor Run Time":10,"Executor CPU Time":9689284,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":50,"Index":19,"Attempt":0,"Launch Time":1547788471383,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":48,"Index":16,"Attempt":0,"Launch Time":1547788471348,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471384,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"13","Value":"684","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"44826580","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":21495808,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":65132,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":14828884,"Value":216122440,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":18,"Value":245,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8682222,"Value":152507182,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":9,"Value":182,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":9,"Executor Deserialize CPU Time":8682222,"Executor Run Time":18,"Executor CPU Time":14828884,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":51,"Index":20,"Attempt":0,"Launch Time":1547788471385,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":49,"Index":18,"Attempt":0,"Launch Time":1547788471364,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471386,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"689","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"45088723","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":21757952,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":68749,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":8110068,"Value":224232508,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":8,"Value":253,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6796299,"Value":159303481,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":189,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":6796299,"Executor Run Time":8,"Executor CPU Time":8110068,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":52,"Index":21,"Attempt":0,"Launch Time":1547788471405,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":51,"Index":20,"Attempt":0,"Launch Time":1547788471385,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471406,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"694","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"45350866","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":22020096,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":72366,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":8359516,"Value":232592024,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":262,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5457354,"Value":164760835,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":194,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5457354,"Executor Run Time":9,"Executor CPU Time":8359516,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":53,"Index":22,"Attempt":0,"Launch Time":1547788471412,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":50,"Index":19,"Attempt":0,"Launch Time":1547788471383,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471412,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"7","Value":"701","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"45613009","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":22282240,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":75983,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":11236782,"Value":243828806,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":11,"Value":273,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7990768,"Value":172751603,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":9,"Value":203,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":9,"Executor Deserialize CPU Time":7990768,"Executor Run Time":11,"Executor CPU Time":11236782,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":54,"Index":23,"Attempt":0,"Launch Time":1547788471426,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":52,"Index":21,"Attempt":0,"Launch Time":1547788471405,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471427,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"706","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"45875152","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":22544384,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":79600,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":8553675,"Value":252382481,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":282,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5575600,"Value":178327203,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":208,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5575600,"Executor Run Time":9,"Executor CPU Time":8553675,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":55,"Index":24,"Attempt":0,"Launch Time":1547788471447,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":53,"Index":22,"Attempt":0,"Launch Time":1547788471412,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471448,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"11","Value":"717","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"46137295","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":22806528,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":83217,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":16255120,"Value":268637601,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":18,"Value":300,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":9055711,"Value":187382914,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":10,"Value":218,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":10,"Executor Deserialize CPU Time":9055711,"Executor Run Time":18,"Executor CPU Time":16255120,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":56,"Index":25,"Attempt":0,"Launch Time":1547788471448,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":54,"Index":23,"Attempt":0,"Launch Time":1547788471426,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471449,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"722","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"46399438","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":23068672,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":86834,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":8880916,"Value":277518517,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":309,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5387499,"Value":192770413,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":223,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5387499,"Executor Run Time":9,"Executor CPU Time":8880916,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":57,"Index":26,"Attempt":0,"Launch Time":1547788471469,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":56,"Index":25,"Attempt":0,"Launch Time":1547788471448,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471469,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"727","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"46661581","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":23330816,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":90451,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":8568476,"Value":286086993,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":318,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5414263,"Value":198184676,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":229,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5414263,"Executor Run Time":9,"Executor CPU Time":8568476,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":58,"Index":27,"Attempt":0,"Launch Time":1547788471490,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":57,"Index":26,"Attempt":0,"Launch Time":1547788471469,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471490,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"732","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"46923724","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":23592960,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":94068,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":8628386,"Value":294715379,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":327,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5388011,"Value":203572687,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":234,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5388011,"Executor Run Time":9,"Executor CPU Time":8628386,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":59,"Index":28,"Attempt":0,"Launch Time":1547788471491,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":55,"Index":24,"Attempt":0,"Launch Time":1547788471447,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471491,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"11","Value":"743","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"47185867","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":23855104,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":97685,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":15818993,"Value":310534372,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":16,"Value":343,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":9904112,"Value":213476799,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":18,"Value":252,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":18,"Executor Deserialize CPU Time":9904112,"Executor Run Time":16,"Executor CPU Time":15818993,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":60,"Index":29,"Attempt":0,"Launch Time":1547788471512,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":58,"Index":27,"Attempt":0,"Launch Time":1547788471490,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471513,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"7","Value":"750","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"47448010","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":24117248,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":101302,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":8960508,"Value":319494880,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":10,"Value":353,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5457103,"Value":218933902,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":258,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5457103,"Executor Run Time":10,"Executor CPU Time":8960508,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":61,"Index":30,"Attempt":0,"Launch Time":1547788471532,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":60,"Index":29,"Attempt":0,"Launch Time":1547788471512,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471532,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"755","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"47710153","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":24379392,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":104919,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":7925873,"Value":327420753,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":362,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4665435,"Value":223599337,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":263,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4665435,"Executor Run Time":9,"Executor CPU Time":7925873,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":62,"Index":31,"Attempt":0,"Launch Time":1547788471536,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":59,"Index":28,"Attempt":0,"Launch Time":1547788471491,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471537,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"10","Value":"765","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"47972296","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":24641536,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":108536,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":16624938,"Value":344045691,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":18,"Value":380,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":13182460,"Value":236781797,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":19,"Value":282,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":19,"Executor Deserialize CPU Time":13182460,"Executor Run Time":18,"Executor CPU Time":16624938,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":63,"Index":32,"Attempt":0,"Launch Time":1547788471557,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":61,"Index":30,"Attempt":0,"Launch Time":1547788471532,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471557,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"7","Value":"772","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"48234439","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":24903680,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":112153,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":11065271,"Value":355110962,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":11,"Value":391,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7259375,"Value":244041172,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":289,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":7259375,"Executor Run Time":11,"Executor CPU Time":11065271,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":64,"Index":33,"Attempt":0,"Launch Time":1547788471575,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":63,"Index":32,"Attempt":0,"Launch Time":1547788471557,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471575,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"776","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"48496582","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":25165824,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":115770,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":7210861,"Value":362321823,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":7,"Value":398,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4726627,"Value":248767799,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":294,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4726627,"Executor Run Time":7,"Executor CPU Time":7210861,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":65,"Index":34,"Attempt":0,"Launch Time":1547788471579,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":62,"Index":31,"Attempt":0,"Launch Time":1547788471536,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471580,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"12","Value":"788","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"48758725","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":25427968,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":119387,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":17941566,"Value":380263389,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":18,"Value":416,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":13630205,"Value":262398004,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":14,"Value":308,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":14,"Executor Deserialize CPU Time":13630205,"Executor Run Time":18,"Executor CPU Time":17941566,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":66,"Index":35,"Attempt":0,"Launch Time":1547788471596,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":64,"Index":33,"Attempt":0,"Launch Time":1547788471575,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471597,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"793","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"49020868","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":25690112,"Internal":true,"Count Failed Values":true},{"ID":345,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3660,"Value":123047,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":8912458,"Value":389175847,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":8,"Value":424,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6097744,"Value":268495748,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":315,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":6097744,"Executor Run Time":8,"Executor CPU Time":8912458,"Result Size":3660,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":67,"Index":36,"Attempt":0,"Launch Time":1547788471617,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":65,"Index":34,"Attempt":0,"Launch Time":1547788471579,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471617,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"13","Value":"806","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"49283011","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":25952256,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":126664,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":15199954,"Value":404375801,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":17,"Value":441,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8754809,"Value":277250557,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":11,"Value":326,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":11,"Executor Deserialize CPU Time":8754809,"Executor Run Time":17,"Executor CPU Time":15199954,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":68,"Index":37,"Attempt":0,"Launch Time":1547788471619,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":66,"Index":35,"Attempt":0,"Launch Time":1547788471596,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471620,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"812","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"49545154","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":26214400,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":130281,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9577896,"Value":413953697,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":450,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5853991,"Value":283104548,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":333,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":5853991,"Executor Run Time":9,"Executor CPU Time":9577896,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":69,"Index":38,"Attempt":0,"Launch Time":1547788471639,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":68,"Index":37,"Attempt":0,"Launch Time":1547788471619,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471640,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"816","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"49807297","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":26476544,"Internal":true,"Count Failed Values":true},{"ID":345,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3660,"Value":133941,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":7837850,"Value":421791547,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":8,"Value":458,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4694415,"Value":287798963,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":337,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4694415,"Executor Run Time":8,"Executor CPU Time":7837850,"Result Size":3660,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":70,"Index":39,"Attempt":0,"Launch Time":1547788471648,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":67,"Index":36,"Attempt":0,"Launch Time":1547788471617,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471649,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"10","Value":"826","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"50069440","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":26738688,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":137558,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":14699497,"Value":436491044,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":15,"Value":473,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8375616,"Value":296174579,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":9,"Value":346,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":9,"Executor Deserialize CPU Time":8375616,"Executor Run Time":15,"Executor CPU Time":14699497,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":71,"Index":40,"Attempt":0,"Launch Time":1547788471657,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":69,"Index":38,"Attempt":0,"Launch Time":1547788471639,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471658,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"830","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"50331583","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":27000832,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":141175,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":7034601,"Value":443525645,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":8,"Value":481,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4375578,"Value":300550157,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":350,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4375578,"Executor Run Time":8,"Executor CPU Time":7034601,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":72,"Index":41,"Attempt":0,"Launch Time":1547788471678,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":71,"Index":40,"Attempt":0,"Launch Time":1547788471657,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471678,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"835","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"50593726","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":27262976,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":144792,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":8094897,"Value":451620542,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":490,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5521327,"Value":306071484,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":356,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5521327,"Executor Run Time":9,"Executor CPU Time":8094897,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":73,"Index":42,"Attempt":0,"Launch Time":1547788471688,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":70,"Index":39,"Attempt":0,"Launch Time":1547788471648,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471688,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"11","Value":"846","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"50855869","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":27525120,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":148409,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":15249188,"Value":466869730,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":20,"Value":510,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8547465,"Value":314618949,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":11,"Value":367,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":11,"Executor Deserialize CPU Time":8547465,"Executor Run Time":20,"Executor CPU Time":15249188,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":74,"Index":43,"Attempt":0,"Launch Time":1547788471696,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":72,"Index":41,"Attempt":0,"Launch Time":1547788471678,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471697,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"851","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"51118012","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":27787264,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":152026,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":8058368,"Value":474928098,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":8,"Value":518,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5166115,"Value":319785064,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":372,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5166115,"Executor Run Time":8,"Executor CPU Time":8058368,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":75,"Index":44,"Attempt":0,"Launch Time":1547788471718,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":73,"Index":42,"Attempt":0,"Launch Time":1547788471688,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471719,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"9","Value":"860","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"51380155","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":28049408,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":155643,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":13338594,"Value":488266692,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":14,"Value":532,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8079370,"Value":327864434,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":9,"Value":381,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":9,"Executor Deserialize CPU Time":8079370,"Executor Run Time":14,"Executor CPU Time":13338594,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":76,"Index":45,"Attempt":0,"Launch Time":1547788471720,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":74,"Index":43,"Attempt":0,"Launch Time":1547788471696,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471720,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"7","Value":"867","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"51642298","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":28311552,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":159260,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9956634,"Value":498223326,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":11,"Value":543,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6060346,"Value":333924780,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":387,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":6060346,"Executor Run Time":11,"Executor CPU Time":9956634,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":77,"Index":46,"Attempt":0,"Launch Time":1547788471737,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":76,"Index":45,"Attempt":0,"Launch Time":1547788471720,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471737,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"871","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"51904441","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":28573696,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":162877,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":7143104,"Value":505366430,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":7,"Value":550,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4396192,"Value":338320972,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":391,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4396192,"Executor Run Time":7,"Executor CPU Time":7143104,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":78,"Index":47,"Attempt":0,"Launch Time":1547788471758,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":75,"Index":44,"Attempt":0,"Launch Time":1547788471718,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471759,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"13","Value":"884","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"52166584","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":28835840,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":166494,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":15062982,"Value":520429412,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":21,"Value":571,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7833118,"Value":346154090,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":9,"Value":400,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":9,"Executor Deserialize CPU Time":7833118,"Executor Run Time":21,"Executor CPU Time":15062982,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":79,"Index":48,"Attempt":0,"Launch Time":1547788471759,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":77,"Index":46,"Attempt":0,"Launch Time":1547788471737,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471759,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"7","Value":"891","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"52428727","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":29097984,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":170111,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":10284571,"Value":530713983,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":580,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6158828,"Value":352312918,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":8,"Value":408,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":8,"Executor Deserialize CPU Time":6158828,"Executor Run Time":9,"Executor CPU Time":10284571,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":80,"Index":49,"Attempt":0,"Launch Time":1547788471780,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":79,"Index":48,"Attempt":0,"Launch Time":1547788471759,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471780,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"897","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"52690870","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":29360128,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":173728,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9330053,"Value":540044036,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":589,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5812350,"Value":358125268,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":414,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5812350,"Executor Run Time":9,"Executor CPU Time":9330053,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":81,"Index":50,"Attempt":0,"Launch Time":1547788471801,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":80,"Index":49,"Attempt":0,"Launch Time":1547788471780,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471802,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"903","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"52953013","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":29622272,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":177345,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9508952,"Value":549552988,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":8,"Value":597,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6126710,"Value":364251978,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":421,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":6126710,"Executor Run Time":8,"Executor CPU Time":9508952,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":82,"Index":51,"Attempt":0,"Launch Time":1547788471803,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":78,"Index":47,"Attempt":0,"Launch Time":1547788471758,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471803,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"9","Value":"912","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"53215156","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":29884416,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":180962,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":13325303,"Value":562878291,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":15,"Value":612,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8203334,"Value":372455312,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":9,"Value":430,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":9,"Executor Deserialize CPU Time":8203334,"Executor Run Time":15,"Executor CPU Time":13325303,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":83,"Index":52,"Attempt":0,"Launch Time":1547788471823,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":81,"Index":50,"Attempt":0,"Launch Time":1547788471801,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471823,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"918","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"53477299","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":30146560,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":184579,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9350207,"Value":572228498,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":621,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5957978,"Value":378413290,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":436,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5957978,"Executor Run Time":9,"Executor CPU Time":9350207,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":84,"Index":53,"Attempt":0,"Launch Time":1547788471844,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":83,"Index":52,"Attempt":0,"Launch Time":1547788471823,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471845,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"924","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"53739442","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":30408704,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":188196,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9388960,"Value":581617458,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":630,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6240504,"Value":384653794,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":442,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":6240504,"Executor Run Time":9,"Executor CPU Time":9388960,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":85,"Index":54,"Attempt":0,"Launch Time":1547788471866,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":84,"Index":53,"Attempt":0,"Launch Time":1547788471844,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471866,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"930","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"54001585","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":30670848,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":191813,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9485397,"Value":591102855,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":10,"Value":640,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5969392,"Value":390623186,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":448,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5969392,"Executor Run Time":10,"Executor CPU Time":9485397,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":86,"Index":55,"Attempt":0,"Launch Time":1547788471886,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":85,"Index":54,"Attempt":0,"Launch Time":1547788471866,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471886,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"935","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"54263728","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":30932992,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":195430,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9059824,"Value":600162679,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":8,"Value":648,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5657365,"Value":396280551,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":454,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5657365,"Executor Run Time":8,"Executor CPU Time":9059824,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":87,"Index":56,"Attempt":0,"Launch Time":1547788471898,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":82,"Index":51,"Attempt":0,"Launch Time":1547788471803,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471898,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"26","Value":"961","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"54525871","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":31195136,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":199047,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":15266506,"Value":615429185,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":42,"Value":690,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8688693,"Value":404969244,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":26,"Value":480,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":26,"Executor Deserialize CPU Time":8688693,"Executor Run Time":42,"Executor CPU Time":15266506,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":88,"Index":57,"Attempt":0,"Launch Time":1547788471908,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":86,"Index":55,"Attempt":0,"Launch Time":1547788471886,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471908,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"967","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"54788014","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":31457280,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":202664,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9852648,"Value":625281833,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":10,"Value":700,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5871602,"Value":410840846,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":486,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5871602,"Executor Run Time":10,"Executor CPU Time":9852648,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":89,"Index":58,"Attempt":0,"Launch Time":1547788471933,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":88,"Index":57,"Attempt":0,"Launch Time":1547788471908,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471934,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"973","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"55050157","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":31719424,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":206281,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9435784,"Value":634717617,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":709,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6260038,"Value":417100884,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":492,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":6260038,"Executor Run Time":9,"Executor CPU Time":9435784,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":90,"Index":59,"Attempt":0,"Launch Time":1547788471955,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":89,"Index":58,"Attempt":0,"Launch Time":1547788471933,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471955,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"978","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"55312300","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":31981568,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":209898,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9084482,"Value":643802099,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":8,"Value":717,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5882081,"Value":422982965,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":498,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5882081,"Executor Run Time":8,"Executor CPU Time":9084482,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":91,"Index":60,"Attempt":0,"Launch Time":1547788471965,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":87,"Index":56,"Attempt":0,"Launch Time":1547788471898,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471965,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"29","Value":"1007","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"55574443","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":227,"Name":"aggregate time total (min, med, max)","Update":"2","Value":"-31","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":32243712,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":213515,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":16055581,"Value":659857680,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":37,"Value":754,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8515827,"Value":431498792,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":17,"Value":515,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":17,"Executor Deserialize CPU Time":8515827,"Executor Run Time":37,"Executor CPU Time":16055581,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":92,"Index":61,"Attempt":0,"Launch Time":1547788471976,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":90,"Index":59,"Attempt":0,"Launch Time":1547788471955,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471976,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"1013","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"55836586","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":32505856,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":217132,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9697936,"Value":669555616,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":763,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5797880,"Value":437296672,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":521,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5797880,"Executor Run Time":9,"Executor CPU Time":9697936,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":93,"Index":62,"Attempt":0,"Launch Time":1547788471995,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":92,"Index":61,"Attempt":0,"Launch Time":1547788471976,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788471995,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1018","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"56098729","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":32768000,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":220749,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":8560813,"Value":678116429,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":772,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5638871,"Value":442935543,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":527,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5638871,"Executor Run Time":9,"Executor CPU Time":8560813,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":94,"Index":63,"Attempt":0,"Launch Time":1547788472000,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":91,"Index":60,"Attempt":0,"Launch Time":1547788471965,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472001,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"17","Value":"1035","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"56360872","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":33030144,"Internal":true,"Count Failed Values":true},{"ID":345,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3660,"Value":224409,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":12009815,"Value":690126244,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":21,"Value":793,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6583189,"Value":449518732,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":8,"Value":535,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":8,"Executor Deserialize CPU Time":6583189,"Executor Run Time":21,"Executor CPU Time":12009815,"Result Size":3660,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":95,"Index":64,"Attempt":0,"Launch Time":1547788472015,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":93,"Index":62,"Attempt":0,"Launch Time":1547788471995,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472015,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"1041","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"56623015","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":33292288,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":228026,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":8550049,"Value":698676293,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":802,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5520058,"Value":455038790,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":540,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5520058,"Executor Run Time":9,"Executor CPU Time":8550049,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":96,"Index":65,"Attempt":0,"Launch Time":1547788472060,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":94,"Index":63,"Attempt":0,"Launch Time":1547788472000,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472061,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"37","Value":"1078","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"56885158","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":33554432,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":231643,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":13494410,"Value":712170703,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":41,"Value":843,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6619804,"Value":461658594,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":547,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":6619804,"Executor Run Time":41,"Executor CPU Time":13494410,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":97,"Index":66,"Attempt":0,"Launch Time":1547788472107,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":95,"Index":64,"Attempt":0,"Launch Time":1547788472015,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472108,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"76","Value":"1154","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"57147301","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":33816576,"Internal":true,"Count Failed Values":true},{"ID":344,"Name":"internal.metrics.jvmGCTime","Update":68,"Value":68,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3660,"Value":235303,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":10897540,"Value":723068243,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":80,"Value":923,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6397648,"Value":468056242,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":553,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":6397648,"Executor Run Time":80,"Executor CPU Time":10897540,"Result Size":3660,"JVM GC Time":68,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":98,"Index":67,"Attempt":0,"Launch Time":1547788472129,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":97,"Index":66,"Attempt":0,"Launch Time":1547788472107,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472130,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1158","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"57409444","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":34078720,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":238920,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":8256923,"Value":731325166,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":10,"Value":933,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6123808,"Value":474180050,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":559,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":6123808,"Executor Run Time":10,"Executor CPU Time":8256923,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":99,"Index":68,"Attempt":0,"Launch Time":1547788472144,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":96,"Index":65,"Attempt":0,"Launch Time":1547788472060,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472144,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"48","Value":"1206","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"57671587","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":34340864,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":242537,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":14766579,"Value":746091745,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":58,"Value":991,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7033307,"Value":481213357,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":17,"Value":576,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":17,"Executor Deserialize CPU Time":7033307,"Executor Run Time":58,"Executor CPU Time":14766579,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":100,"Index":69,"Attempt":0,"Launch Time":1547788472146,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":98,"Index":67,"Attempt":0,"Launch Time":1547788472129,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472146,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1210","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"57933730","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":34603008,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":246154,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":7143208,"Value":753234953,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":8,"Value":999,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4411266,"Value":485624623,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":580,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4411266,"Executor Run Time":8,"Executor CPU Time":7143208,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":101,"Index":70,"Attempt":0,"Launch Time":1547788472163,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":100,"Index":69,"Attempt":0,"Launch Time":1547788472146,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472164,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1214","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"58195873","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":34865152,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":249771,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":6949656,"Value":760184609,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":6,"Value":1005,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4403849,"Value":490028472,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":585,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4403849,"Executor Run Time":6,"Executor CPU Time":6949656,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":102,"Index":71,"Attempt":0,"Launch Time":1547788472178,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":99,"Index":68,"Attempt":0,"Launch Time":1547788472144,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472179,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"9","Value":"1223","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"58458016","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":35127296,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":253388,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":12131790,"Value":772316399,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":13,"Value":1018,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7962672,"Value":497991144,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":11,"Value":596,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":11,"Executor Deserialize CPU Time":7962672,"Executor Run Time":13,"Executor CPU Time":12131790,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":103,"Index":72,"Attempt":0,"Launch Time":1547788472180,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":101,"Index":70,"Attempt":0,"Launch Time":1547788472163,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472180,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1227","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"58720159","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":35389440,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":257005,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":6978432,"Value":779294831,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":8,"Value":1026,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4476822,"Value":502467966,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":600,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4476822,"Executor Run Time":8,"Executor CPU Time":6978432,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":104,"Index":73,"Attempt":0,"Launch Time":1547788472196,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":103,"Index":72,"Attempt":0,"Launch Time":1547788472180,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472196,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1231","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"58982302","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":35651584,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":260622,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":6807151,"Value":786101982,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":8,"Value":1034,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4341850,"Value":506809816,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":603,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":4341850,"Executor Run Time":8,"Executor CPU Time":6807151,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":105,"Index":74,"Attempt":0,"Launch Time":1547788472204,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":102,"Index":71,"Attempt":0,"Launch Time":1547788472178,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472204,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"7","Value":"1238","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"59244445","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":35913728,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":264239,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":10728505,"Value":796830487,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":12,"Value":1046,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6101014,"Value":512910830,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":609,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":6101014,"Executor Run Time":12,"Executor CPU Time":10728505,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":106,"Index":75,"Attempt":0,"Launch Time":1547788472211,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":104,"Index":73,"Attempt":0,"Launch Time":1547788472196,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472212,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1241","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"59506588","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":36175872,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":267856,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":6550817,"Value":803381304,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":7,"Value":1053,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4359486,"Value":517270316,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":613,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4359486,"Executor Run Time":7,"Executor CPU Time":6550817,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":107,"Index":76,"Attempt":0,"Launch Time":1547788472224,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":105,"Index":74,"Attempt":0,"Launch Time":1547788472204,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472225,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"1247","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"59768731","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":36438016,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":271473,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9032531,"Value":812413835,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":1062,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4621545,"Value":521891861,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":618,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4621545,"Executor Run Time":9,"Executor CPU Time":9032531,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":108,"Index":77,"Attempt":0,"Launch Time":1547788472228,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":106,"Index":75,"Attempt":0,"Launch Time":1547788472211,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472228,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1250","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"60030874","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":36700160,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":275090,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":6409712,"Value":818823547,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":6,"Value":1068,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4324233,"Value":526216094,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":623,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4324233,"Executor Run Time":6,"Executor CPU Time":6409712,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":109,"Index":78,"Attempt":0,"Launch Time":1547788472243,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":108,"Index":77,"Attempt":0,"Launch Time":1547788472228,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472243,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1253","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"60293017","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":36962304,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":278707,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":5948893,"Value":824772440,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":7,"Value":1075,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4157099,"Value":530373193,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":626,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":4157099,"Executor Run Time":7,"Executor CPU Time":5948893,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":110,"Index":79,"Attempt":0,"Launch Time":1547788472255,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":107,"Index":76,"Attempt":0,"Launch Time":1547788472224,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472256,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"9","Value":"1262","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"60555160","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":227,"Name":"aggregate time total (min, med, max)","Update":"0","Value":"-49","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":37224448,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":282324,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":12000030,"Value":836772470,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":13,"Value":1088,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6168677,"Value":536541870,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":633,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":6168677,"Executor Run Time":13,"Executor CPU Time":12000030,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":111,"Index":80,"Attempt":0,"Launch Time":1547788472260,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":109,"Index":78,"Attempt":0,"Launch Time":1547788472243,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472260,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1265","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"60817303","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":37486592,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":285941,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":6510742,"Value":843283212,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":7,"Value":1095,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4303260,"Value":540845130,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":636,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":4303260,"Executor Run Time":7,"Executor CPU Time":6510742,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":112,"Index":81,"Attempt":0,"Launch Time":1547788472276,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":111,"Index":80,"Attempt":0,"Launch Time":1547788472260,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472277,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1269","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"61079446","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":37748736,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":289558,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":6908036,"Value":850191248,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":6,"Value":1101,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4403845,"Value":545248975,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":641,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4403845,"Executor Run Time":6,"Executor CPU Time":6908036,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":113,"Index":82,"Attempt":0,"Launch Time":1547788472286,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":110,"Index":79,"Attempt":0,"Launch Time":1547788472255,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472286,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"9","Value":"1278","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"61341589","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":38010880,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":293175,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":12129290,"Value":862320538,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":14,"Value":1115,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7920563,"Value":553169538,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":8,"Value":649,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":8,"Executor Deserialize CPU Time":7920563,"Executor Run Time":14,"Executor CPU Time":12129290,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":114,"Index":83,"Attempt":0,"Launch Time":1547788472291,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":112,"Index":81,"Attempt":0,"Launch Time":1547788472276,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472291,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1281","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"61603732","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":38273024,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":296792,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":6066655,"Value":868387193,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":6,"Value":1121,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3671418,"Value":556840956,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":653,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3671418,"Executor Run Time":6,"Executor CPU Time":6066655,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":115,"Index":84,"Attempt":0,"Launch Time":1547788472305,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":114,"Index":83,"Attempt":0,"Launch Time":1547788472291,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472306,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1284","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"61865875","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":38535168,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":300409,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":5658380,"Value":874045573,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":6,"Value":1127,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3797079,"Value":560638035,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":656,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3797079,"Executor Run Time":6,"Executor CPU Time":5658380,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":116,"Index":85,"Attempt":0,"Launch Time":1547788472310,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":113,"Index":82,"Attempt":0,"Launch Time":1547788472286,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472311,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"1290","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"62128018","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":38797312,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":304026,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":10268198,"Value":884313771,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":10,"Value":1137,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6569994,"Value":567208029,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":9,"Value":665,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":9,"Executor Deserialize CPU Time":6569994,"Executor Run Time":10,"Executor CPU Time":10268198,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":117,"Index":86,"Attempt":0,"Launch Time":1547788472323,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":115,"Index":84,"Attempt":0,"Launch Time":1547788472305,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472324,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1293","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"62390161","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":39059456,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":307643,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":6761333,"Value":891075104,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":7,"Value":1144,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4522127,"Value":571730156,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":670,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4522127,"Executor Run Time":7,"Executor CPU Time":6761333,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":118,"Index":87,"Attempt":0,"Launch Time":1547788472341,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":117,"Index":86,"Attempt":0,"Launch Time":1547788472323,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472342,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1297","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"62652304","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":39321600,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":311260,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":7963864,"Value":899038968,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":8,"Value":1152,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5032419,"Value":576762575,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":675,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5032419,"Executor Run Time":8,"Executor CPU Time":7963864,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":119,"Index":88,"Attempt":0,"Launch Time":1547788472359,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":116,"Index":85,"Attempt":0,"Launch Time":1547788472310,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472360,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"23","Value":"1320","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"62914447","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":39583744,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":314877,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":10779655,"Value":909818623,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":29,"Value":1181,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6727712,"Value":583490287,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":13,"Value":688,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":13,"Executor Deserialize CPU Time":6727712,"Executor Run Time":29,"Executor CPU Time":10779655,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":120,"Index":89,"Attempt":0,"Launch Time":1547788472363,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":118,"Index":87,"Attempt":0,"Launch Time":1547788472341,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472363,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1325","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"63176590","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":39845888,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":318494,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":7516058,"Value":917334681,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":9,"Value":1190,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4921440,"Value":588411727,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":692,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4921440,"Executor Run Time":9,"Executor CPU Time":7516058,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":121,"Index":90,"Attempt":0,"Launch Time":1547788472381,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":120,"Index":89,"Attempt":0,"Launch Time":1547788472363,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472382,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1329","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"63438733","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":40108032,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":322111,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":7885694,"Value":925220375,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":8,"Value":1198,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4935183,"Value":593346910,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":697,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4935183,"Executor Run Time":8,"Executor CPU Time":7885694,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":122,"Index":91,"Attempt":0,"Launch Time":1547788472382,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":119,"Index":88,"Attempt":0,"Launch Time":1547788472359,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472383,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1333","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"63700876","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":40370176,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":325728,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":8096868,"Value":933317243,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":8,"Value":1206,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5212304,"Value":598559214,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":703,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5212304,"Executor Run Time":8,"Executor CPU Time":8096868,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":123,"Index":92,"Attempt":0,"Launch Time":1547788472403,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":121,"Index":90,"Attempt":0,"Launch Time":1547788472381,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472404,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1337","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"63963019","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":40632320,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":329345,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":7567228,"Value":940884471,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":8,"Value":1214,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5508158,"Value":604067372,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":9,"Value":712,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":9,"Executor Deserialize CPU Time":5508158,"Executor Run Time":8,"Executor CPU Time":7567228,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":124,"Index":93,"Attempt":0,"Launch Time":1547788472409,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":122,"Index":91,"Attempt":0,"Launch Time":1547788472382,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472409,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"8","Value":"1345","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"64225162","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":40894464,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":332962,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":10648027,"Value":951532498,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":12,"Value":1226,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5917859,"Value":609985231,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":8,"Value":720,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":8,"Executor Deserialize CPU Time":5917859,"Executor Run Time":12,"Executor CPU Time":10648027,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":125,"Index":94,"Attempt":0,"Launch Time":1547788472419,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":123,"Index":92,"Attempt":0,"Launch Time":1547788472403,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472419,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1349","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"64487305","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":41156608,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":336579,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":6875526,"Value":958408024,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":8,"Value":1234,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3210347,"Value":613195578,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":723,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3210347,"Executor Run Time":8,"Executor CPU Time":6875526,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":126,"Index":95,"Attempt":0,"Launch Time":1547788472438,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":124,"Index":93,"Attempt":0,"Launch Time":1547788472409,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472438,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"1355","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"64749448","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":41418752,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":340196,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9613993,"Value":968022017,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":12,"Value":1246,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6005946,"Value":619201524,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":729,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":6005946,"Executor Run Time":12,"Executor CPU Time":9613993,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":127,"Index":96,"Attempt":0,"Launch Time":1547788472440,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":125,"Index":94,"Attempt":0,"Launch Time":1547788472419,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472441,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"1361","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"65011591","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":41680896,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":343813,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":8756332,"Value":976778349,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":10,"Value":1256,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5992726,"Value":625194250,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":735,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5992726,"Executor Run Time":10,"Executor CPU Time":8756332,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":128,"Index":97,"Attempt":0,"Launch Time":1547788472457,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":127,"Index":96,"Attempt":0,"Launch Time":1547788472440,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472458,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1365","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"65273734","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":41943040,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":347430,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":7036673,"Value":983815022,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":7,"Value":1263,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3641036,"Value":628835286,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":738,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3641036,"Executor Run Time":7,"Executor CPU Time":7036673,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":129,"Index":98,"Attempt":0,"Launch Time":1547788472460,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":126,"Index":95,"Attempt":0,"Launch Time":1547788472438,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472461,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1370","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"65535877","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":42205184,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":351047,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":8455047,"Value":992270069,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":10,"Value":1273,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6254404,"Value":635089690,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":744,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":6254404,"Executor Run Time":10,"Executor CPU Time":8455047,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Info":{"Task ID":130,"Index":99,"Attempt":0,"Launch Time":1547788472474,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":128,"Index":97,"Attempt":0,"Launch Time":1547788472457,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472474,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1374","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"65798020","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":42467328,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":354664,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":7315691,"Value":999585760,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":6,"Value":1279,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3708854,"Value":638798544,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":749,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":3708854,"Executor Run Time":6,"Executor CPU Time":7315691,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":129,"Index":98,"Attempt":0,"Launch Time":1547788472460,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472488,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"9","Value":"1383","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"66060163","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":42729472,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":358281,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":9576011,"Value":1009161771,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":13,"Value":1292,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5162911,"Value":643961455,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":8,"Value":757,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":8,"Executor Deserialize CPU Time":5162911,"Executor Run Time":13,"Executor CPU Time":9576011,"Result Size":3617,"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":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":130,"Index":99,"Attempt":0,"Launch Time":1547788472474,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472489,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1386","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"66322306","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":60,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":42991616,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Update":3617,"Value":361898,"Internal":true,"Count Failed Values":true},{"ID":342,"Name":"internal.metrics.executorCpuTime","Update":5697446,"Value":1014859217,"Internal":true,"Count Failed Values":true},{"ID":341,"Name":"internal.metrics.executorRunTime","Update":6,"Value":1298,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3959798,"Value":647921253,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":761,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3959798,"Executor Run Time":6,"Executor CPU Time":5697446,"Result Size":3617,"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":"SparkListenerStageCompleted","Stage Info":{"Stage ID":12,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":100,"RDD Info":[{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"50\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[11],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788471074,"Completion Time":1547788472490,"Accumulables":[{"ID":227,"Name":"aggregate time total (min, med, max)","Value":"-49","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":209,"Name":"records read","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":355,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":340,"Name":"internal.metrics.executorDeserializeCpuTime","Value":647921253,"Internal":true,"Count Failed Values":true},{"ID":343,"Name":"internal.metrics.resultSize","Value":361898,"Internal":true,"Count Failed Values":true},{"ID":352,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":223,"Name":"duration total (min, med, max)","Value":"1386","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":345,"Name":"internal.metrics.resultSerializationTime","Value":3,"Internal":true,"Count Failed Values":true},{"ID":354,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":60,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.peakExecutionMemory","Value":42991616,"Internal":true,"Count Failed Values":true},{"ID":339,"Name":"internal.metrics.executorDeserializeTime","Value":761,"Internal":true,"Count Failed Values":true},{"ID":225,"Name":"peak memory total (min, med, max)","Value":"66322306","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":342,"Name":"internal.metrics.executorCpuTime","Value":1014859217,"Internal":true,"Count Failed Values":true},{"ID":207,"Name":"local bytes read total (min, med, max)","Value":"96","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":351,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":228,"Name":"avg hash probe (min, med, max)","Value":"20","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":204,"Name":"local blocks read","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":341,"Name":"internal.metrics.executorRunTime","Value":1298,"Internal":true,"Count Failed Values":true},{"ID":350,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":344,"Name":"internal.metrics.jvmGCTime","Value":68,"Internal":true,"Count Failed Values":true},{"ID":353,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":356,"Name":"internal.metrics.shuffle.read.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":224,"Name":"number of output rows","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"}]}} -{"Event":"SparkListenerJobEnd","Job ID":8,"Completion Time":1547788472491,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerJobStart","Job ID":9,"Submission Time":1547788472506,"Stage Infos":[{"Stage ID":13,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":1,"RDD Info":[{"RDD ID":21,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":19,"Name":"FileScanRDD","Scope":"{\"id\":\"46\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"46\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]},{"Stage ID":14,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":75,"RDD Info":[{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"50\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[13],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[13,14],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"4","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":14,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":75,"RDD Info":[{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"50\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[13],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788472512,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"4","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerTaskStart","Stage ID":14,"Stage Attempt ID":0,"Task Info":{"Task ID":131,"Index":43,"Attempt":0,"Launch Time":1547788472520,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":14,"Stage Attempt ID":0,"Task Info":{"Task ID":132,"Index":0,"Attempt":0,"Launch Time":1547788472521,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":14,"Stage Attempt ID":0,"Task Info":{"Task ID":133,"Index":1,"Attempt":0,"Launch Time":1547788472557,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":131,"Index":43,"Attempt":0,"Launch Time":1547788472520,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472558,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"11","Value":"1397","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":228,"Name":"avg hash probe (min, med, max)","Update":"10","Value":"30","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":224,"Name":"number of output rows","Update":"1","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"17039359","Value":"83361665","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":227,"Name":"aggregate time total (min, med, max)","Update":"8","Value":"-63","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":204,"Name":"local blocks read","Update":"1","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":207,"Name":"local bytes read total (min, med, max)","Update":"62","Value":"59","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":209,"Name":"records read","Update":"1","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":63,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":17039360,"Value":17039360,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3656,"Value":3656,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":15381200,"Value":15381200,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":16,"Value":16,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8872982,"Value":8872982,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":15,"Value":15,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":15,"Executor Deserialize CPU Time":8872982,"Executor Run Time":16,"Executor CPU Time":15381200,"Result Size":3656,"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":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":63,"Total Records Read":1},"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":134,"Index":2,"Attempt":0,"Launch Time":1547788472568,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":132,"Index":0,"Attempt":0,"Launch Time":1547788472521,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472568,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1402","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"83623808","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":17301504,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":7273,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":8909017,"Value":24290217,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":9,"Value":25,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":16079042,"Value":24952024,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":32,"Value":47,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":32,"Executor Deserialize CPU Time":16079042,"Executor Run Time":9,"Executor CPU Time":8909017,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":135,"Index":3,"Attempt":0,"Launch Time":1547788472571,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":133,"Index":1,"Attempt":0,"Launch Time":1547788472557,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472571,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1405","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"83885951","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":17563648,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":10890,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5219491,"Value":29509708,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":5,"Value":30,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3278056,"Value":28230080,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":51,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3278056,"Executor Run Time":5,"Executor CPU Time":5219491,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":136,"Index":4,"Attempt":0,"Launch Time":1547788472584,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":135,"Index":3,"Attempt":0,"Launch Time":1547788472571,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472585,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1408","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"84148094","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":17825792,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":14507,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":6013191,"Value":35522899,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":6,"Value":36,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3216254,"Value":31446334,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":54,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3216254,"Executor Run Time":6,"Executor CPU Time":6013191,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":137,"Index":5,"Attempt":0,"Launch Time":1547788472598,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":136,"Index":4,"Attempt":0,"Launch Time":1547788472584,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472598,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1411","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"84410237","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":18087936,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":18124,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5480342,"Value":41003241,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":5,"Value":41,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3527925,"Value":34974259,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":58,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3527925,"Executor Run Time":5,"Executor CPU Time":5480342,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":138,"Index":6,"Attempt":0,"Launch Time":1547788472600,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":134,"Index":2,"Attempt":0,"Launch Time":1547788472568,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472601,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"1417","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"84672380","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":18350080,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":21741,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":9627469,"Value":50630710,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":10,"Value":51,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5302230,"Value":40276489,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":8,"Value":66,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":8,"Executor Deserialize CPU Time":5302230,"Executor Run Time":10,"Executor CPU Time":9627469,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":139,"Index":7,"Attempt":0,"Launch Time":1547788472611,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":137,"Index":5,"Attempt":0,"Launch Time":1547788472598,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472612,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"2","Value":"1419","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"84934523","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":18612224,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":25358,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5276321,"Value":55907031,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":5,"Value":56,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3785216,"Value":44061705,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":70,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3785216,"Executor Run Time":5,"Executor CPU Time":5276321,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":140,"Index":8,"Attempt":0,"Launch Time":1547788472626,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":139,"Index":7,"Attempt":0,"Launch Time":1547788472611,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472627,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1422","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"85196666","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":18874368,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":28975,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":6176643,"Value":62083674,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":6,"Value":62,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3761399,"Value":47823104,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":74,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3761399,"Executor Run Time":6,"Executor CPU Time":6176643,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":141,"Index":9,"Attempt":0,"Launch Time":1547788472641,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":138,"Index":6,"Attempt":0,"Launch Time":1547788472600,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472642,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"25","Value":"1447","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"85458809","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":227,"Name":"aggregate time total (min, med, max)","Update":"4","Value":"-66","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":19136512,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":32592,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":12602772,"Value":74686446,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":28,"Value":90,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4746241,"Value":52569345,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":80,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":4746241,"Executor Run Time":28,"Executor CPU Time":12602772,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":142,"Index":10,"Attempt":0,"Launch Time":1547788472644,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":140,"Index":8,"Attempt":0,"Launch Time":1547788472626,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472644,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1451","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"85720952","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":19398656,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":36209,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7771104,"Value":82457550,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":8,"Value":98,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4715570,"Value":57284915,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":85,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4715570,"Executor Run Time":8,"Executor CPU Time":7771104,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":143,"Index":11,"Attempt":0,"Launch Time":1547788472657,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":142,"Index":10,"Attempt":0,"Launch Time":1547788472644,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472657,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"2","Value":"1453","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"85983095","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":19660800,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":39826,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5452707,"Value":87910257,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":6,"Value":104,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3625734,"Value":60910649,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":88,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3625734,"Executor Run Time":6,"Executor CPU Time":5452707,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":144,"Index":12,"Attempt":0,"Launch Time":1547788472671,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":143,"Index":11,"Attempt":0,"Launch Time":1547788472657,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472671,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"2","Value":"1455","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"86245238","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":19922944,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":43443,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5300134,"Value":93210391,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":6,"Value":110,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3612247,"Value":64522896,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":91,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3612247,"Executor Run Time":6,"Executor CPU Time":5300134,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":145,"Index":13,"Attempt":0,"Launch Time":1547788472678,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":141,"Index":9,"Attempt":0,"Launch Time":1547788472641,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472678,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"12","Value":"1467","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"86507381","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":20185088,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":47060,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":10008577,"Value":103218968,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":18,"Value":128,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6689362,"Value":71212258,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":98,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":6689362,"Executor Run Time":18,"Executor CPU Time":10008577,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":146,"Index":14,"Attempt":0,"Launch Time":1547788472684,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":144,"Index":12,"Attempt":0,"Launch Time":1547788472671,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472684,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"2","Value":"1469","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"86769524","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":20447232,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":50677,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5118993,"Value":108337961,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":5,"Value":133,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3530194,"Value":74742452,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":102,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3530194,"Executor Run Time":5,"Executor CPU Time":5118993,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":147,"Index":15,"Attempt":0,"Launch Time":1547788472696,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":145,"Index":13,"Attempt":0,"Launch Time":1547788472678,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472696,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1474","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"87031667","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":20709376,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":54294,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7866688,"Value":116204649,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":8,"Value":141,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4784844,"Value":79527296,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":107,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4784844,"Executor Run Time":8,"Executor CPU Time":7866688,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":148,"Index":16,"Attempt":0,"Launch Time":1547788472699,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":146,"Index":14,"Attempt":0,"Launch Time":1547788472684,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472699,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"2","Value":"1476","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"87293810","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":20971520,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":57911,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5552266,"Value":121756915,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":5,"Value":146,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4023055,"Value":83550351,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":112,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4023055,"Executor Run Time":5,"Executor CPU Time":5552266,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":149,"Index":17,"Attempt":0,"Launch Time":1547788472712,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":148,"Index":16,"Attempt":0,"Launch Time":1547788472699,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472713,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"2","Value":"1478","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"87555953","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":21233664,"Internal":true,"Count Failed Values":true},{"ID":370,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3660,"Value":61571,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5358929,"Value":127115844,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":5,"Value":151,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3396464,"Value":86946815,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":115,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3396464,"Executor Run Time":5,"Executor CPU Time":5358929,"Result Size":3660,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":150,"Index":18,"Attempt":0,"Launch Time":1547788472716,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":147,"Index":15,"Attempt":0,"Launch Time":1547788472696,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472716,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1483","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"87818096","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":21495808,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":65188,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":8292843,"Value":135408687,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":9,"Value":160,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5069544,"Value":92016359,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":121,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5069544,"Executor Run Time":9,"Executor CPU Time":8292843,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":151,"Index":19,"Attempt":0,"Launch Time":1547788472726,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":149,"Index":17,"Attempt":0,"Launch Time":1547788472712,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472727,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1486","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"88080239","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":21757952,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":68805,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5812644,"Value":141221331,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":6,"Value":166,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4242098,"Value":96258457,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":125,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4242098,"Executor Run Time":6,"Executor CPU Time":5812644,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":152,"Index":20,"Attempt":0,"Launch Time":1547788472732,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":150,"Index":18,"Attempt":0,"Launch Time":1547788472716,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472732,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1490","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"88342382","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":22020096,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":72422,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":6952671,"Value":148174002,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":8,"Value":174,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4692829,"Value":100951286,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":129,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4692829,"Executor Run Time":8,"Executor CPU Time":6952671,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":153,"Index":21,"Attempt":0,"Launch Time":1547788472745,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":151,"Index":19,"Attempt":0,"Launch Time":1547788472726,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472745,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1494","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"88604525","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":22282240,"Internal":true,"Count Failed Values":true},{"ID":370,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3660,"Value":76082,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7842932,"Value":156016934,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":8,"Value":182,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5765915,"Value":106717201,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":134,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5765915,"Executor Run Time":8,"Executor CPU Time":7842932,"Result Size":3660,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":154,"Index":22,"Attempt":0,"Launch Time":1547788472749,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":152,"Index":20,"Attempt":0,"Launch Time":1547788472732,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472750,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1498","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"88866668","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":22544384,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":79699,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7300900,"Value":163317834,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":7,"Value":189,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4355555,"Value":111072756,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":139,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4355555,"Executor Run Time":7,"Executor CPU Time":7300900,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":155,"Index":23,"Attempt":0,"Launch Time":1547788472762,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":153,"Index":21,"Attempt":0,"Launch Time":1547788472745,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472763,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1502","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"89128811","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":22806528,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":83316,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7418972,"Value":170736806,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":8,"Value":197,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5372143,"Value":116444899,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":144,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5372143,"Executor Run Time":8,"Executor CPU Time":7418972,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":156,"Index":24,"Attempt":0,"Launch Time":1547788472768,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":154,"Index":22,"Attempt":0,"Launch Time":1547788472749,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472769,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1506","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"89390954","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":23068672,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":86933,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7366926,"Value":178103732,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":9,"Value":206,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5480706,"Value":121925605,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":149,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5480706,"Executor Run Time":9,"Executor CPU Time":7366926,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":157,"Index":25,"Attempt":0,"Launch Time":1547788472777,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":155,"Index":23,"Attempt":0,"Launch Time":1547788472762,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472778,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"2","Value":"1508","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"89653097","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":23330816,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":90550,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5710660,"Value":183814392,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":6,"Value":212,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4584705,"Value":126510310,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":153,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4584705,"Executor Run Time":6,"Executor CPU Time":5710660,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":158,"Index":26,"Attempt":0,"Launch Time":1547788472791,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":157,"Index":25,"Attempt":0,"Launch Time":1547788472777,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472792,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1511","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"89915240","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":23592960,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":94167,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5788208,"Value":189602600,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":5,"Value":217,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3542437,"Value":130052747,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":157,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3542437,"Executor Run Time":5,"Executor CPU Time":5788208,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":159,"Index":27,"Attempt":0,"Launch Time":1547788472792,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":156,"Index":24,"Attempt":0,"Launch Time":1547788472768,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472793,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"1517","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"90177383","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":23855104,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":97784,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":9339029,"Value":198941629,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":10,"Value":227,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5172685,"Value":135225432,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":164,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":5172685,"Executor Run Time":10,"Executor CPU Time":9339029,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":160,"Index":28,"Attempt":0,"Launch Time":1547788472806,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":158,"Index":26,"Attempt":0,"Launch Time":1547788472791,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472807,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1520","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"90439526","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":24117248,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":101401,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5736880,"Value":204678509,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":6,"Value":233,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3653261,"Value":138878693,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":168,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3653261,"Executor Run Time":6,"Executor CPU Time":5736880,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":161,"Index":29,"Attempt":0,"Launch Time":1547788472815,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":159,"Index":27,"Attempt":0,"Launch Time":1547788472792,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472815,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1525","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"90701669","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":24379392,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":105018,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7310374,"Value":211988883,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":12,"Value":245,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5092058,"Value":143970751,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":173,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5092058,"Executor Run Time":12,"Executor CPU Time":7310374,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":162,"Index":30,"Attempt":0,"Launch Time":1547788472820,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":160,"Index":28,"Attempt":0,"Launch Time":1547788472806,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472821,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"2","Value":"1527","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"90963812","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":24641536,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":108635,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5302259,"Value":217291142,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":5,"Value":250,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3681762,"Value":147652513,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":177,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3681762,"Executor Run Time":5,"Executor CPU Time":5302259,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":163,"Index":31,"Attempt":0,"Launch Time":1547788472834,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":162,"Index":30,"Attempt":0,"Launch Time":1547788472820,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472835,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1530","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"91225955","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":24903680,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":112252,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5629506,"Value":222920648,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":5,"Value":255,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3404257,"Value":151056770,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":181,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3404257,"Executor Run Time":5,"Executor CPU Time":5629506,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":164,"Index":32,"Attempt":0,"Launch Time":1547788472839,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":161,"Index":29,"Attempt":0,"Launch Time":1547788472815,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472839,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1534","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"91488098","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":25165824,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":115869,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7223660,"Value":230144308,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":7,"Value":262,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4732379,"Value":155789149,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":8,"Value":189,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":8,"Executor Deserialize CPU Time":4732379,"Executor Run Time":7,"Executor CPU Time":7223660,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":165,"Index":33,"Attempt":0,"Launch Time":1547788472848,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":163,"Index":31,"Attempt":0,"Launch Time":1547788472834,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472849,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1537","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"91750241","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":25427968,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":119486,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5762476,"Value":235906784,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":6,"Value":268,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3458234,"Value":159247383,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":192,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3458234,"Executor Run Time":6,"Executor CPU Time":5762476,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":166,"Index":34,"Attempt":0,"Launch Time":1547788472858,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":164,"Index":32,"Attempt":0,"Launch Time":1547788472839,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472859,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1541","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"92012384","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":25690112,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":123103,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7748774,"Value":243655558,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":8,"Value":276,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5373685,"Value":164621068,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":198,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5373685,"Executor Run Time":8,"Executor CPU Time":7748774,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":167,"Index":35,"Attempt":0,"Launch Time":1547788472863,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":165,"Index":33,"Attempt":0,"Launch Time":1547788472848,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472863,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1544","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"92274527","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":25952256,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":126720,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5611576,"Value":249267134,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":5,"Value":281,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3702884,"Value":168323952,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":202,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3702884,"Executor Run Time":5,"Executor CPU Time":5611576,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":168,"Index":36,"Attempt":0,"Launch Time":1547788472877,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":167,"Index":35,"Attempt":0,"Launch Time":1547788472863,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472878,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1547","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"92536670","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":26214400,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":130337,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":6093576,"Value":255360710,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":6,"Value":287,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3726766,"Value":172050718,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":205,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3726766,"Executor Run Time":6,"Executor CPU Time":6093576,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":169,"Index":37,"Attempt":0,"Launch Time":1547788472878,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":166,"Index":34,"Attempt":0,"Launch Time":1547788472858,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472878,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1552","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"92798813","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":26476544,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":133954,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7716743,"Value":263077453,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":8,"Value":295,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4871405,"Value":176922123,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":210,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4871405,"Executor Run Time":8,"Executor CPU Time":7716743,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":170,"Index":38,"Attempt":0,"Launch Time":1547788472903,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":168,"Index":36,"Attempt":0,"Launch Time":1547788472877,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472904,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1556","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"93060956","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":26738688,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":137571,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":8759637,"Value":271837090,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":8,"Value":303,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5927866,"Value":182849989,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":217,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":5927866,"Executor Run Time":8,"Executor CPU Time":8759637,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":171,"Index":39,"Attempt":0,"Launch Time":1547788472917,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":170,"Index":38,"Attempt":0,"Launch Time":1547788472903,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472917,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"2","Value":"1558","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"93323099","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":27000832,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":141188,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5642002,"Value":277479092,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":5,"Value":308,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3540817,"Value":186390806,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":221,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3540817,"Executor Run Time":5,"Executor CPU Time":5642002,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":172,"Index":40,"Attempt":0,"Launch Time":1547788472935,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":171,"Index":39,"Attempt":0,"Launch Time":1547788472917,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472936,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1562","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"93585242","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":27262976,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":144805,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7661206,"Value":285140298,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":8,"Value":316,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5642762,"Value":192033568,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":226,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5642762,"Executor Run Time":8,"Executor CPU Time":7661206,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":173,"Index":41,"Attempt":0,"Launch Time":1547788472948,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":172,"Index":40,"Attempt":0,"Launch Time":1547788472935,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472948,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"2","Value":"1564","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"93847385","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":27525120,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":148422,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5302931,"Value":290443229,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":5,"Value":321,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3436848,"Value":195470416,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":229,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3436848,"Executor Run Time":5,"Executor CPU Time":5302931,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":174,"Index":42,"Attempt":0,"Launch Time":1547788472961,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":173,"Index":41,"Attempt":0,"Launch Time":1547788472948,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472961,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1567","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"94109528","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":27787264,"Internal":true,"Count Failed Values":true},{"ID":370,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3660,"Value":152082,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5739070,"Value":296182299,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":5,"Value":326,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3562405,"Value":199032821,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":233,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3562405,"Executor Run Time":5,"Executor CPU Time":5739070,"Result Size":3660,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":175,"Index":44,"Attempt":0,"Launch Time":1547788472976,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":174,"Index":42,"Attempt":0,"Launch Time":1547788472961,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472976,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1570","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"94371671","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":28049408,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":155699,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5963709,"Value":302146008,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":6,"Value":332,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3636041,"Value":202668862,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":236,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3636041,"Executor Run Time":6,"Executor CPU Time":5963709,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":176,"Index":45,"Attempt":0,"Launch Time":1547788472992,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":175,"Index":44,"Attempt":0,"Launch Time":1547788472976,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788472993,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1573","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"94633814","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":28311552,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":159316,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":6565669,"Value":308711677,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":7,"Value":339,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5404627,"Value":208073489,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":241,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5404627,"Executor Run Time":7,"Executor CPU Time":6565669,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":177,"Index":46,"Attempt":0,"Launch Time":1547788473009,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":176,"Index":45,"Attempt":0,"Launch Time":1547788472992,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473009,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1577","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"94895957","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":28573696,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":162933,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7033349,"Value":315745026,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":6,"Value":345,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5074273,"Value":213147762,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":247,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5074273,"Executor Run Time":6,"Executor CPU Time":7033349,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":178,"Index":47,"Attempt":0,"Launch Time":1547788473026,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":177,"Index":46,"Attempt":0,"Launch Time":1547788473009,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473026,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1581","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"95158100","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":28835840,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":166550,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7390998,"Value":323136024,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":8,"Value":353,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4966515,"Value":218114277,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":251,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4966515,"Executor Run Time":8,"Executor CPU Time":7390998,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":179,"Index":48,"Attempt":0,"Launch Time":1547788473032,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":169,"Index":37,"Attempt":0,"Launch Time":1547788472878,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473032,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1586","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"95420243","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":29097984,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":170167,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7742581,"Value":330878605,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":8,"Value":361,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4684364,"Value":222798641,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":256,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4684364,"Executor Run Time":8,"Executor CPU Time":7742581,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":180,"Index":49,"Attempt":0,"Launch Time":1547788473044,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":178,"Index":47,"Attempt":0,"Launch Time":1547788473026,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473045,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1590","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"95682386","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":29360128,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":173784,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7257991,"Value":338136596,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":8,"Value":369,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5307271,"Value":228105912,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":261,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5307271,"Executor Run Time":8,"Executor CPU Time":7257991,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":181,"Index":50,"Attempt":0,"Launch Time":1547788473052,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":179,"Index":48,"Attempt":0,"Launch Time":1547788473032,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473052,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1595","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"95944529","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":29622272,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":177401,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":8473342,"Value":346609938,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":9,"Value":378,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4873441,"Value":232979353,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":266,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4873441,"Executor Run Time":9,"Executor CPU Time":8473342,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":182,"Index":51,"Attempt":0,"Launch Time":1547788473065,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":180,"Index":49,"Attempt":0,"Launch Time":1547788473044,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473065,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"1601","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"96206672","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":29884416,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":181018,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":9595563,"Value":356205501,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":10,"Value":388,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5409968,"Value":238389321,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":271,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5409968,"Executor Run Time":10,"Executor CPU Time":9595563,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":183,"Index":52,"Attempt":0,"Launch Time":1547788473077,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":181,"Index":50,"Attempt":0,"Launch Time":1547788473052,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473077,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1606","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"96468815","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":30146560,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":184635,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":8216175,"Value":364421676,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":8,"Value":396,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":12085600,"Value":250474921,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":12,"Value":283,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":12,"Executor Deserialize CPU Time":12085600,"Executor Run Time":8,"Executor CPU Time":8216175,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":184,"Index":53,"Attempt":0,"Launch Time":1547788473087,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":182,"Index":51,"Attempt":0,"Launch Time":1547788473065,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473087,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"1612","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"96730958","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":30408704,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":188252,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":9744028,"Value":374165704,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":10,"Value":406,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7027353,"Value":257502274,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":290,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":7027353,"Executor Run Time":10,"Executor CPU Time":9744028,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":185,"Index":54,"Attempt":0,"Launch Time":1547788473098,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":183,"Index":52,"Attempt":0,"Launch Time":1547788473077,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473099,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"1618","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"96993101","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":30670848,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":191869,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7420363,"Value":381586067,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":9,"Value":415,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4053871,"Value":261556145,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":294,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4053871,"Executor Run Time":9,"Executor CPU Time":7420363,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":186,"Index":55,"Attempt":0,"Launch Time":1547788473108,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":184,"Index":53,"Attempt":0,"Launch Time":1547788473087,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473109,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1623","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"97255244","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":30932992,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":195486,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":8678513,"Value":390264580,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":9,"Value":424,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6978523,"Value":268534668,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":301,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":6978523,"Executor Run Time":9,"Executor CPU Time":8678513,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":187,"Index":56,"Attempt":0,"Launch Time":1547788473117,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":185,"Index":54,"Attempt":0,"Launch Time":1547788473098,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473117,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1628","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"97517387","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":31195136,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":199103,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7960536,"Value":398225116,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":7,"Value":431,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4604373,"Value":273139041,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":307,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":4604373,"Executor Run Time":7,"Executor CPU Time":7960536,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":188,"Index":57,"Attempt":0,"Launch Time":1547788473135,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":187,"Index":56,"Attempt":0,"Launch Time":1547788473117,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473136,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1633","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"97779530","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":31457280,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":202720,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":8225881,"Value":406450997,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":8,"Value":439,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4682601,"Value":277821642,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":312,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4682601,"Executor Run Time":8,"Executor CPU Time":8225881,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":189,"Index":58,"Attempt":0,"Launch Time":1547788473137,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":186,"Index":55,"Attempt":0,"Launch Time":1547788473108,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473137,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"1639","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"98041673","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":31719424,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":206337,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":9838423,"Value":416289420,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":11,"Value":450,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7232048,"Value":285053690,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":8,"Value":320,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":8,"Executor Deserialize CPU Time":7232048,"Executor Run Time":11,"Executor CPU Time":9838423,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":190,"Index":59,"Attempt":0,"Launch Time":1547788473153,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":188,"Index":57,"Attempt":0,"Launch Time":1547788473135,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473153,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1644","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"98303816","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":31981568,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":209954,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":8000257,"Value":424289677,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":7,"Value":457,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4745183,"Value":289798873,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":325,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4745183,"Executor Run Time":7,"Executor CPU Time":8000257,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":191,"Index":60,"Attempt":0,"Launch Time":1547788473158,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":189,"Index":58,"Attempt":0,"Launch Time":1547788473137,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473158,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"1650","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"98565959","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":32243712,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":213571,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":8960938,"Value":433250615,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":8,"Value":465,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6215634,"Value":296014507,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":332,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":6215634,"Executor Run Time":8,"Executor CPU Time":8960938,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":192,"Index":61,"Attempt":0,"Launch Time":1547788473176,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":190,"Index":59,"Attempt":0,"Launch Time":1547788473153,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473177,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1655","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"98828102","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":32505856,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":217188,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":8143954,"Value":441394569,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":9,"Value":474,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4799811,"Value":300814318,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":337,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4799811,"Executor Run Time":9,"Executor CPU Time":8143954,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":193,"Index":62,"Attempt":0,"Launch Time":1547788473179,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":191,"Index":60,"Attempt":0,"Launch Time":1547788473158,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473180,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1660","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"99090245","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":32768000,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":220805,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":8037261,"Value":449431830,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":9,"Value":483,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7208632,"Value":308022950,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":344,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":7208632,"Executor Run Time":9,"Executor CPU Time":8037261,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":194,"Index":63,"Attempt":0,"Launch Time":1547788473198,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":192,"Index":61,"Attempt":0,"Launch Time":1547788473176,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473198,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"1666","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"99352388","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":33030144,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":224422,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":8361505,"Value":457793335,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":9,"Value":492,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5062107,"Value":313085057,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":351,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":5062107,"Executor Run Time":9,"Executor CPU Time":8361505,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":195,"Index":64,"Attempt":0,"Launch Time":1547788473200,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":193,"Index":62,"Attempt":0,"Launch Time":1547788473179,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473201,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"1672","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"99614531","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":33292288,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":228039,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":9530500,"Value":467323835,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":10,"Value":502,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5959460,"Value":319044517,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":356,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":5959460,"Executor Run Time":10,"Executor CPU Time":9530500,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":196,"Index":65,"Attempt":0,"Launch Time":1547788473220,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":195,"Index":64,"Attempt":0,"Launch Time":1547788473200,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473220,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1676","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"99876674","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":33554432,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":231656,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7830036,"Value":475153871,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":8,"Value":510,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5342625,"Value":324387142,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":362,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5342625,"Executor Run Time":8,"Executor CPU Time":7830036,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":197,"Index":66,"Attempt":0,"Launch Time":1547788473234,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":194,"Index":63,"Attempt":0,"Launch Time":1547788473198,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473235,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"15","Value":"1691","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"100138817","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":33816576,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":235273,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":12073310,"Value":487227181,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":20,"Value":530,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5305109,"Value":329692251,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":369,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":5305109,"Executor Run Time":20,"Executor CPU Time":12073310,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":198,"Index":67,"Attempt":0,"Launch Time":1547788473239,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":196,"Index":65,"Attempt":0,"Launch Time":1547788473220,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473240,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1696","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"100400960","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":34078720,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":238890,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":8041348,"Value":495268529,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":8,"Value":538,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5597838,"Value":335290089,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":375,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5597838,"Executor Run Time":8,"Executor CPU Time":8041348,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":199,"Index":68,"Attempt":0,"Launch Time":1547788473259,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":197,"Index":66,"Attempt":0,"Launch Time":1547788473234,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473260,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"10","Value":"1706","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"100663103","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":34340864,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":242507,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":10310150,"Value":505578679,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":13,"Value":551,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":5294912,"Value":340585001,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":381,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":5294912,"Executor Run Time":13,"Executor CPU Time":10310150,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":200,"Index":69,"Attempt":0,"Launch Time":1547788473267,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":198,"Index":67,"Attempt":0,"Launch Time":1547788473239,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473267,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"1712","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"100925246","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":34603008,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":246124,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":11581666,"Value":517160345,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":13,"Value":564,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7177709,"Value":347762710,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":388,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":7177709,"Executor Run Time":13,"Executor CPU Time":11581666,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":201,"Index":70,"Attempt":0,"Launch Time":1547788473279,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":199,"Index":68,"Attempt":0,"Launch Time":1547788473259,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473279,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"5","Value":"1717","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"101187389","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":34865152,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":249741,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":8196414,"Value":525356759,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":9,"Value":573,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4995663,"Value":352758373,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":394,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":4995663,"Executor Run Time":9,"Executor CPU Time":8196414,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":202,"Index":71,"Attempt":0,"Launch Time":1547788473294,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":200,"Index":69,"Attempt":0,"Launch Time":1547788473267,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473294,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"6","Value":"1723","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"101449532","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":35127296,"Internal":true,"Count Failed Values":true},{"ID":370,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":4,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3660,"Value":253401,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":11710046,"Value":537066805,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":12,"Value":585,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":9483809,"Value":362242182,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":9,"Value":403,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":9,"Executor Deserialize CPU Time":9483809,"Executor Run Time":12,"Executor CPU Time":11710046,"Result Size":3660,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":203,"Index":72,"Attempt":0,"Launch Time":1547788473297,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":201,"Index":70,"Attempt":0,"Launch Time":1547788473279,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473298,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"4","Value":"1727","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"101711675","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":35389440,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":257018,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":6968674,"Value":544035479,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":7,"Value":592,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4770014,"Value":367012196,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":409,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":4770014,"Executor Run Time":7,"Executor CPU Time":6968674,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":204,"Index":73,"Attempt":0,"Launch Time":1547788473312,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":202,"Index":71,"Attempt":0,"Launch Time":1547788473294,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473312,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1730","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"101973818","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":35651584,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":260635,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":6861059,"Value":550896538,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":7,"Value":599,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7050527,"Value":374062723,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":7,"Value":416,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":7,"Executor Deserialize CPU Time":7050527,"Executor Run Time":7,"Executor CPU Time":6861059,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Info":{"Task ID":205,"Index":74,"Attempt":0,"Launch Time":1547788473317,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":203,"Index":72,"Attempt":0,"Launch Time":1547788473297,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473318,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"7","Value":"1737","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"102235961","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":35913728,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":264252,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":7897659,"Value":558794197,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":11,"Value":610,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4454965,"Value":378517688,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":421,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4454965,"Executor Run Time":11,"Executor CPU Time":7897659,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":204,"Index":73,"Attempt":0,"Launch Time":1547788473312,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473326,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"2","Value":"1739","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"102498104","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":36175872,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":267869,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":5088294,"Value":563882491,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":4,"Value":614,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4281314,"Value":382799002,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":426,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4281314,"Executor Run Time":4,"Executor CPU Time":5088294,"Result Size":3617,"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":14,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":205,"Index":74,"Attempt":0,"Launch Time":1547788473317,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473334,"Failed":false,"Killed":false,"Accumulables":[{"ID":223,"Name":"duration total (min, med, max)","Update":"3","Value":"1742","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":225,"Name":"peak memory total (min, med, max)","Update":"262143","Value":"102760247","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":63,"Internal":true,"Count Failed Values":true},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Update":262144,"Value":36438016,"Internal":true,"Count Failed Values":true},{"ID":368,"Name":"internal.metrics.resultSize","Update":3617,"Value":271486,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Update":6293560,"Value":570176051,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Update":6,"Value":620,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4340534,"Value":387139536,"Internal":true,"Count Failed Values":true},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":431,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4340534,"Executor Run Time":6,"Executor CPU Time":6293560,"Result Size":3617,"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":"SparkListenerStageCompleted","Stage Info":{"Stage ID":14,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:110","Number of Tasks":75,"RDD Info":[{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"50\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"49\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"45\",\"name\":\"Exchange\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"42\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:110","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[13],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:110)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788472512,"Completion Time":1547788473335,"Accumulables":[{"ID":227,"Name":"aggregate time total (min, med, max)","Value":"-66","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":209,"Name":"records read","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":364,"Name":"internal.metrics.executorDeserializeTime","Value":431,"Internal":true,"Count Failed Values":true},{"ID":373,"Name":"internal.metrics.peakExecutionMemory","Value":36438016,"Internal":true,"Count Failed Values":true},{"ID":367,"Name":"internal.metrics.executorCpuTime","Value":570176051,"Internal":true,"Count Failed Values":true},{"ID":376,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":1,"Internal":true,"Count Failed Values":true},{"ID":370,"Name":"internal.metrics.resultSerializationTime","Value":4,"Internal":true,"Count Failed Values":true},{"ID":379,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":63,"Internal":true,"Count Failed Values":true},{"ID":223,"Name":"duration total (min, med, max)","Value":"1742","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":378,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":381,"Name":"internal.metrics.shuffle.read.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":375,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":366,"Name":"internal.metrics.executorRunTime","Value":620,"Internal":true,"Count Failed Values":true},{"ID":225,"Name":"peak memory total (min, med, max)","Value":"102760247","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":207,"Name":"local bytes read total (min, med, max)","Value":"59","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":228,"Name":"avg hash probe (min, med, max)","Value":"30","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":204,"Name":"local blocks read","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":368,"Name":"internal.metrics.resultSize","Value":271486,"Internal":true,"Count Failed Values":true},{"ID":377,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":380,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":365,"Name":"internal.metrics.executorDeserializeCpuTime","Value":387139536,"Internal":true,"Count Failed Values":true},{"ID":224,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"}]}} -{"Event":"SparkListenerJobEnd","Job ID":9,"Completion Time":1547788473336,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":4,"time":1547788473342} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":5,"description":"createOrReplaceTempView at SparkSQLExample.scala:122","details":"org.apache.spark.sql.Dataset.createOrReplaceTempView(Dataset.scala:3021)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:122)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\nCreateViewCommand `people`, false, true, LocalTempView\n +- Relation[age#7L,name#8] json\n\n== Analyzed Logical Plan ==\n\nCreateViewCommand `people`, false, true, LocalTempView\n +- Relation[age#7L,name#8] json\n\n== Optimized Logical Plan ==\nCreateViewCommand `people`, false, true, LocalTempView\n +- Relation[age#7L,name#8] json\n\n== Physical Plan ==\nExecute CreateViewCommand\n +- CreateViewCommand `people`, false, true, LocalTempView\n +- Relation[age#7L,name#8] json\n","sparkPlanInfo":{"nodeName":"Execute CreateViewCommand","simpleString":"Execute CreateViewCommand","children":[],"metadata":{},"metrics":[]},"time":1547788473539} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":5,"time":1547788473541} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":6,"description":"show at SparkSQLExample.scala:125","details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:125)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#7L as string) AS age#76, cast(name#8 as string) AS name#77]\n +- Project [age#7L, name#8]\n +- SubqueryAlias `people`\n +- Relation[age#7L,name#8] json\n\n== Analyzed Logical Plan ==\nage: string, name: string\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#7L as string) AS age#76, cast(name#8 as string) AS name#77]\n +- Project [age#7L, name#8]\n +- SubqueryAlias `people`\n +- Relation[age#7L,name#8] json\n\n== Optimized Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#7L as string) AS age#76, name#8]\n +- Relation[age#7L,name#8] json\n\n== Physical Plan ==\nCollectLimit 21\n+- *(1) Project [cast(age#7L as string) AS age#76, name#8]\n +- *(1) FileScan json [age#7L,name#8] Batched: false, DataFilters: [], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n","sparkPlanInfo":{"nodeName":"CollectLimit","simpleString":"CollectLimit 21","children":[{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"Project","simpleString":"Project [cast(age#7L as string) AS age#76, name#8]","children":[{"nodeName":"Scan json ","simpleString":"FileScan json [age#7L,name#8] Batched: false, DataFilters: [], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct","children":[],"metadata":{"Location":"InMemoryFileIndex[hdfs://node6:8020/user/spark/examples/src/main/resources/people.json]","ReadSchema":"struct","Format":"JSON","Batched":"false","PartitionFilters":"[]","PushedFilters":"[]","DataFilters":"[]"},"metrics":[{"name":"number of output rows","accumulatorId":400,"metricType":"sum"},{"name":"number of files","accumulatorId":401,"metricType":"sum"},{"name":"metadata time","accumulatorId":402,"metricType":"sum"},{"name":"scan time total (min, med, max)","accumulatorId":403,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration total (min, med, max)","accumulatorId":399,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":397,"metricType":"sum"},{"name":"shuffle write time total (min, med, max)","accumulatorId":398,"metricType":"nsTiming"},{"name":"records read","accumulatorId":395,"metricType":"sum"},{"name":"local bytes read total (min, med, max)","accumulatorId":393,"metricType":"size"},{"name":"fetch wait time total (min, med, max)","accumulatorId":394,"metricType":"timing"},{"name":"remote bytes read total (min, med, max)","accumulatorId":391,"metricType":"size"},{"name":"local blocks read","accumulatorId":390,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":389,"metricType":"sum"},{"name":"remote bytes read to disk total (min, med, max)","accumulatorId":392,"metricType":"size"},{"name":"shuffle bytes written total (min, med, max)","accumulatorId":396,"metricType":"size"}]},"time":1547788473656} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":6,"accumUpdates":[[401,1],[402,0]]} -{"Event":"SparkListenerJobStart","Job ID":10,"Submission Time":1547788473735,"Stage Infos":[{"Stage ID":15,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:125","Number of Tasks":1,"RDD Info":[{"RDD ID":29,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"66\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:125","Parent IDs":[28],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":28,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"65\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:125","Parent IDs":[27],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":26,"Name":"FileScanRDD","Scope":"{\"id\":\"62\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:125","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":27,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"62\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:125","Parent IDs":[26],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:125)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[15],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"6","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":15,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:125","Number of Tasks":1,"RDD Info":[{"RDD ID":29,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"66\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:125","Parent IDs":[28],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":28,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"65\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:125","Parent IDs":[27],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":26,"Name":"FileScanRDD","Scope":"{\"id\":\"62\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:125","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":27,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"62\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:125","Parent IDs":[26],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:125)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788473736,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"6","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerTaskStart","Stage ID":15,"Stage Attempt ID":0,"Task Info":{"Task ID":206,"Index":0,"Attempt":0,"Launch Time":1547788473742,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":15,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":206,"Index":0,"Attempt":0,"Launch Time":1547788473742,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473812,"Failed":false,"Killed":false,"Accumulables":[{"ID":399,"Name":"duration total (min, med, max)","Update":"45","Value":"44","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":400,"Name":"number of output rows","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":426,"Name":"internal.metrics.input.recordsRead","Update":3,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":425,"Name":"internal.metrics.input.bytesRead","Update":73,"Value":73,"Internal":true,"Count Failed Values":true},{"ID":408,"Name":"internal.metrics.resultSize","Update":1229,"Value":1229,"Internal":true,"Count Failed Values":true},{"ID":407,"Name":"internal.metrics.executorCpuTime","Update":37928680,"Value":37928680,"Internal":true,"Count Failed Values":true},{"ID":406,"Name":"internal.metrics.executorRunTime","Update":49,"Value":49,"Internal":true,"Count Failed Values":true},{"ID":405,"Name":"internal.metrics.executorDeserializeCpuTime","Update":10668425,"Value":10668425,"Internal":true,"Count Failed Values":true},{"ID":404,"Name":"internal.metrics.executorDeserializeTime","Update":15,"Value":15,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":15,"Executor Deserialize CPU Time":10668425,"Executor Run Time":49,"Executor CPU Time":37928680,"Result Size":1229,"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":73,"Records Read":3},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":15,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:125","Number of Tasks":1,"RDD Info":[{"RDD ID":29,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"66\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:125","Parent IDs":[28],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":28,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"65\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:125","Parent IDs":[27],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":26,"Name":"FileScanRDD","Scope":"{\"id\":\"62\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:125","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":27,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"62\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:125","Parent IDs":[26],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:125)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788473736,"Completion Time":1547788473813,"Accumulables":[{"ID":406,"Name":"internal.metrics.executorRunTime","Value":49,"Internal":true,"Count Failed Values":true},{"ID":400,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":405,"Name":"internal.metrics.executorDeserializeCpuTime","Value":10668425,"Internal":true,"Count Failed Values":true},{"ID":399,"Name":"duration total (min, med, max)","Value":"44","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":426,"Name":"internal.metrics.input.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":408,"Name":"internal.metrics.resultSize","Value":1229,"Internal":true,"Count Failed Values":true},{"ID":404,"Name":"internal.metrics.executorDeserializeTime","Value":15,"Internal":true,"Count Failed Values":true},{"ID":425,"Name":"internal.metrics.input.bytesRead","Value":73,"Internal":true,"Count Failed Values":true},{"ID":407,"Name":"internal.metrics.executorCpuTime","Value":37928680,"Internal":true,"Count Failed Values":true}]}} -{"Event":"SparkListenerJobEnd","Job ID":10,"Completion Time":1547788473813,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":6,"time":1547788473815} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":7,"description":"createGlobalTempView at SparkSQLExample.scala:137","details":"org.apache.spark.sql.Dataset.createGlobalTempView(Dataset.scala:3040)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:137)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\nCreateViewCommand `people`, false, false, GlobalTempView\n +- Relation[age#7L,name#8] json\n\n== Analyzed Logical Plan ==\n\nCreateViewCommand `people`, false, false, GlobalTempView\n +- Relation[age#7L,name#8] json\n\n== Optimized Logical Plan ==\nCreateViewCommand `people`, false, false, GlobalTempView\n +- Relation[age#7L,name#8] json\n\n== Physical Plan ==\nExecute CreateViewCommand\n +- CreateViewCommand `people`, false, false, GlobalTempView\n +- Relation[age#7L,name#8] json\n","sparkPlanInfo":{"nodeName":"Execute CreateViewCommand","simpleString":"Execute CreateViewCommand","children":[],"metadata":{},"metrics":[]},"time":1547788473824} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":7,"time":1547788473824} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":8,"description":"show at SparkSQLExample.scala:140","details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:140)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#7L as string) AS age#89, cast(name#8 as string) AS name#90]\n +- Project [age#7L, name#8]\n +- SubqueryAlias `global_temp`.`people`\n +- Relation[age#7L,name#8] json\n\n== Analyzed Logical Plan ==\nage: string, name: string\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#7L as string) AS age#89, cast(name#8 as string) AS name#90]\n +- Project [age#7L, name#8]\n +- SubqueryAlias `global_temp`.`people`\n +- Relation[age#7L,name#8] json\n\n== Optimized Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#7L as string) AS age#89, name#8]\n +- Relation[age#7L,name#8] json\n\n== Physical Plan ==\nCollectLimit 21\n+- *(1) Project [cast(age#7L as string) AS age#89, name#8]\n +- *(1) FileScan json [age#7L,name#8] Batched: false, DataFilters: [], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n","sparkPlanInfo":{"nodeName":"CollectLimit","simpleString":"CollectLimit 21","children":[{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"Project","simpleString":"Project [cast(age#7L as string) AS age#89, name#8]","children":[{"nodeName":"Scan json ","simpleString":"FileScan json [age#7L,name#8] Batched: false, DataFilters: [], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct","children":[],"metadata":{"Location":"InMemoryFileIndex[hdfs://node6:8020/user/spark/examples/src/main/resources/people.json]","ReadSchema":"struct","Format":"JSON","Batched":"false","PartitionFilters":"[]","PushedFilters":"[]","DataFilters":"[]"},"metrics":[{"name":"number of output rows","accumulatorId":440,"metricType":"sum"},{"name":"number of files","accumulatorId":441,"metricType":"sum"},{"name":"metadata time","accumulatorId":442,"metricType":"sum"},{"name":"scan time total (min, med, max)","accumulatorId":443,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration total (min, med, max)","accumulatorId":439,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":437,"metricType":"sum"},{"name":"shuffle write time total (min, med, max)","accumulatorId":438,"metricType":"nsTiming"},{"name":"records read","accumulatorId":435,"metricType":"sum"},{"name":"local bytes read total (min, med, max)","accumulatorId":433,"metricType":"size"},{"name":"fetch wait time total (min, med, max)","accumulatorId":434,"metricType":"timing"},{"name":"remote bytes read total (min, med, max)","accumulatorId":431,"metricType":"size"},{"name":"local blocks read","accumulatorId":430,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":429,"metricType":"sum"},{"name":"remote bytes read to disk total (min, med, max)","accumulatorId":432,"metricType":"size"},{"name":"shuffle bytes written total (min, med, max)","accumulatorId":436,"metricType":"size"}]},"time":1547788473855} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":8,"accumUpdates":[[441,1],[442,0]]} -{"Event":"SparkListenerJobStart","Job ID":11,"Submission Time":1547788473893,"Stage Infos":[{"Stage ID":16,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:140","Number of Tasks":1,"RDD Info":[{"RDD ID":33,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"75\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:140","Parent IDs":[32],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":32,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"74\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:140","Parent IDs":[31],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":31,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"71\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:140","Parent IDs":[30],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":30,"Name":"FileScanRDD","Scope":"{\"id\":\"71\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:140","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:140)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[16],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"8","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":16,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:140","Number of Tasks":1,"RDD Info":[{"RDD ID":33,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"75\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:140","Parent IDs":[32],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":32,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"74\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:140","Parent IDs":[31],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":31,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"71\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:140","Parent IDs":[30],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":30,"Name":"FileScanRDD","Scope":"{\"id\":\"71\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:140","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:140)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788473894,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"8","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerTaskStart","Stage ID":16,"Stage Attempt ID":0,"Task Info":{"Task ID":207,"Index":0,"Attempt":0,"Launch Time":1547788473904,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":16,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":207,"Index":0,"Attempt":0,"Launch Time":1547788473904,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788473959,"Failed":false,"Killed":false,"Accumulables":[{"ID":439,"Name":"duration total (min, med, max)","Update":"36","Value":"35","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":440,"Name":"number of output rows","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":466,"Name":"internal.metrics.input.recordsRead","Update":3,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":465,"Name":"internal.metrics.input.bytesRead","Update":73,"Value":73,"Internal":true,"Count Failed Values":true},{"ID":448,"Name":"internal.metrics.resultSize","Update":1229,"Value":1229,"Internal":true,"Count Failed Values":true},{"ID":447,"Name":"internal.metrics.executorCpuTime","Update":31052290,"Value":31052290,"Internal":true,"Count Failed Values":true},{"ID":446,"Name":"internal.metrics.executorRunTime","Update":38,"Value":38,"Internal":true,"Count Failed Values":true},{"ID":445,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6114378,"Value":6114378,"Internal":true,"Count Failed Values":true},{"ID":444,"Name":"internal.metrics.executorDeserializeTime","Update":12,"Value":12,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":12,"Executor Deserialize CPU Time":6114378,"Executor Run Time":38,"Executor CPU Time":31052290,"Result Size":1229,"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":73,"Records Read":3},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":16,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:140","Number of Tasks":1,"RDD Info":[{"RDD ID":33,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"75\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:140","Parent IDs":[32],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":32,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"74\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:140","Parent IDs":[31],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":31,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"71\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:140","Parent IDs":[30],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":30,"Name":"FileScanRDD","Scope":"{\"id\":\"71\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:140","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:140)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788473894,"Completion Time":1547788473960,"Accumulables":[{"ID":445,"Name":"internal.metrics.executorDeserializeCpuTime","Value":6114378,"Internal":true,"Count Failed Values":true},{"ID":439,"Name":"duration total (min, med, max)","Value":"35","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":448,"Name":"internal.metrics.resultSize","Value":1229,"Internal":true,"Count Failed Values":true},{"ID":465,"Name":"internal.metrics.input.bytesRead","Value":73,"Internal":true,"Count Failed Values":true},{"ID":447,"Name":"internal.metrics.executorCpuTime","Value":31052290,"Internal":true,"Count Failed Values":true},{"ID":444,"Name":"internal.metrics.executorDeserializeTime","Value":12,"Internal":true,"Count Failed Values":true},{"ID":446,"Name":"internal.metrics.executorRunTime","Value":38,"Internal":true,"Count Failed Values":true},{"ID":440,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":466,"Name":"internal.metrics.input.recordsRead","Value":3,"Internal":true,"Count Failed Values":true}]}} -{"Event":"SparkListenerJobEnd","Job ID":11,"Completion Time":1547788473960,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":8,"time":1547788473961} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":9,"description":"show at SparkSQLExample.scala:150","details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:150)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#7L as string) AS age#102, cast(name#8 as string) AS name#103]\n +- Project [age#7L, name#8]\n +- SubqueryAlias `global_temp`.`people`\n +- Relation[age#7L,name#8] json\n\n== Analyzed Logical Plan ==\nage: string, name: string\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#7L as string) AS age#102, cast(name#8 as string) AS name#103]\n +- Project [age#7L, name#8]\n +- SubqueryAlias `global_temp`.`people`\n +- Relation[age#7L,name#8] json\n\n== Optimized Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#7L as string) AS age#102, name#8]\n +- Relation[age#7L,name#8] json\n\n== Physical Plan ==\nCollectLimit 21\n+- *(1) Project [cast(age#7L as string) AS age#102, name#8]\n +- *(1) FileScan json [age#7L,name#8] Batched: false, DataFilters: [], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n","sparkPlanInfo":{"nodeName":"CollectLimit","simpleString":"CollectLimit 21","children":[{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"Project","simpleString":"Project [cast(age#7L as string) AS age#102, name#8]","children":[{"nodeName":"Scan json ","simpleString":"FileScan json [age#7L,name#8] Batched: false, DataFilters: [], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct","children":[],"metadata":{"Location":"InMemoryFileIndex[hdfs://node6:8020/user/spark/examples/src/main/resources/people.json]","ReadSchema":"struct","Format":"JSON","Batched":"false","PartitionFilters":"[]","PushedFilters":"[]","DataFilters":"[]"},"metrics":[{"name":"number of output rows","accumulatorId":480,"metricType":"sum"},{"name":"number of files","accumulatorId":481,"metricType":"sum"},{"name":"metadata time","accumulatorId":482,"metricType":"sum"},{"name":"scan time total (min, med, max)","accumulatorId":483,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration total (min, med, max)","accumulatorId":479,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":477,"metricType":"sum"},{"name":"shuffle write time total (min, med, max)","accumulatorId":478,"metricType":"nsTiming"},{"name":"records read","accumulatorId":475,"metricType":"sum"},{"name":"local bytes read total (min, med, max)","accumulatorId":473,"metricType":"size"},{"name":"fetch wait time total (min, med, max)","accumulatorId":474,"metricType":"timing"},{"name":"remote bytes read total (min, med, max)","accumulatorId":471,"metricType":"size"},{"name":"local blocks read","accumulatorId":470,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":469,"metricType":"sum"},{"name":"remote bytes read to disk total (min, med, max)","accumulatorId":472,"metricType":"size"},{"name":"shuffle bytes written total (min, med, max)","accumulatorId":476,"metricType":"size"}]},"time":1547788473996} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":9,"accumUpdates":[[481,1],[482,0]]} -{"Event":"SparkListenerJobStart","Job ID":12,"Submission Time":1547788474036,"Stage Infos":[{"Stage ID":17,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:150","Number of Tasks":1,"RDD Info":[{"RDD ID":37,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"84\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:150","Parent IDs":[36],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":34,"Name":"FileScanRDD","Scope":"{\"id\":\"80\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:150","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":36,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"83\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:150","Parent IDs":[35],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":35,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"80\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:150","Parent IDs":[34],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:150)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[17],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"9","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":17,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:150","Number of Tasks":1,"RDD Info":[{"RDD ID":37,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"84\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:150","Parent IDs":[36],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":34,"Name":"FileScanRDD","Scope":"{\"id\":\"80\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:150","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":36,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"83\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:150","Parent IDs":[35],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":35,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"80\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:150","Parent IDs":[34],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:150)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788474037,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"9","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerTaskStart","Stage ID":17,"Stage Attempt ID":0,"Task Info":{"Task ID":208,"Index":0,"Attempt":0,"Launch Time":1547788474043,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":17,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":208,"Index":0,"Attempt":0,"Launch Time":1547788474043,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788474109,"Failed":false,"Killed":false,"Accumulables":[{"ID":479,"Name":"duration total (min, med, max)","Update":"47","Value":"46","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":480,"Name":"number of output rows","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":506,"Name":"internal.metrics.input.recordsRead","Update":3,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":505,"Name":"internal.metrics.input.bytesRead","Update":73,"Value":73,"Internal":true,"Count Failed Values":true},{"ID":488,"Name":"internal.metrics.resultSize","Update":1229,"Value":1229,"Internal":true,"Count Failed Values":true},{"ID":487,"Name":"internal.metrics.executorCpuTime","Update":41699265,"Value":41699265,"Internal":true,"Count Failed Values":true},{"ID":486,"Name":"internal.metrics.executorRunTime","Update":50,"Value":50,"Internal":true,"Count Failed Values":true},{"ID":485,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7235198,"Value":7235198,"Internal":true,"Count Failed Values":true},{"ID":484,"Name":"internal.metrics.executorDeserializeTime","Update":12,"Value":12,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":12,"Executor Deserialize CPU Time":7235198,"Executor Run Time":50,"Executor CPU Time":41699265,"Result Size":1229,"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":73,"Records Read":3},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":17,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:150","Number of Tasks":1,"RDD Info":[{"RDD ID":37,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"84\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:150","Parent IDs":[36],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":34,"Name":"FileScanRDD","Scope":"{\"id\":\"80\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:150","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":36,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"83\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:150","Parent IDs":[35],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":35,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"80\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:150","Parent IDs":[34],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:150)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788474037,"Completion Time":1547788474110,"Accumulables":[{"ID":505,"Name":"internal.metrics.input.bytesRead","Value":73,"Internal":true,"Count Failed Values":true},{"ID":487,"Name":"internal.metrics.executorCpuTime","Value":41699265,"Internal":true,"Count Failed Values":true},{"ID":480,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":486,"Name":"internal.metrics.executorRunTime","Value":50,"Internal":true,"Count Failed Values":true},{"ID":479,"Name":"duration total (min, med, max)","Value":"46","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":506,"Name":"internal.metrics.input.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":488,"Name":"internal.metrics.resultSize","Value":1229,"Internal":true,"Count Failed Values":true},{"ID":485,"Name":"internal.metrics.executorDeserializeCpuTime","Value":7235198,"Internal":true,"Count Failed Values":true},{"ID":484,"Name":"internal.metrics.executorDeserializeTime","Value":12,"Internal":true,"Count Failed Values":true}]}} -{"Event":"SparkListenerJobEnd","Job ID":12,"Completion Time":1547788474111,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":9,"time":1547788474112} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":10,"description":"show at SparkSQLExample.scala:166","details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runDatasetCreationExample(SparkSQLExample.scala:166)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:50)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(name#113 as string) AS name#120, cast(age#114L as string) AS age#121]\n +- LocalRelation [name#113, age#114L]\n\n== Analyzed Logical Plan ==\nname: string, age: string\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(name#113 as string) AS name#120, cast(age#114L as string) AS age#121]\n +- LocalRelation [name#113, age#114L]\n\n== Optimized Logical Plan ==\nLocalRelation [name#120, age#121]\n\n== Physical Plan ==\nLocalTableScan [name#120, age#121]\n","sparkPlanInfo":{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [name#120, age#121]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":509,"metricType":"sum"}]},"time":1547788474280} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":10,"time":1547788474298} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":11,"description":"collect at SparkSQLExample.scala:175","details":"org.apache.spark.sql.Dataset.collect(Dataset.scala:2708)\norg.apache.spark.examples.sql.SparkSQLExample$.runDatasetCreationExample(SparkSQLExample.scala:175)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:50)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\n'SerializeFromObject [input[0, int, false] AS value#134]\n+- 'MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2489/1993254639@5aaa636f, int, [StructField(value,IntegerType,false)], obj#133: int\n +- 'DeserializeToObject unresolveddeserializer(assertnotnull(upcast(getcolumnbyordinal(0, IntegerType), IntegerType, - root class: \"scala.Int\"))), obj#132: int\n +- LocalRelation [value#128]\n\n== Analyzed Logical Plan ==\nvalue: int\nSerializeFromObject [input[0, int, false] AS value#134]\n+- MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2489/1993254639@5aaa636f, int, [StructField(value,IntegerType,false)], obj#133: int\n +- DeserializeToObject assertnotnull(cast(value#128 as int)), obj#132: int\n +- LocalRelation [value#128]\n\n== Optimized Logical Plan ==\nSerializeFromObject [input[0, int, false] AS value#134]\n+- MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2489/1993254639@5aaa636f, int, [StructField(value,IntegerType,false)], obj#133: int\n +- DeserializeToObject value#128: int, obj#132: int\n +- LocalRelation [value#128]\n\n== Physical Plan ==\n*(1) SerializeFromObject [input[0, int, false] AS value#134]\n+- *(1) MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2489/1993254639@5aaa636f, obj#133: int\n +- *(1) DeserializeToObject value#128: int, obj#132: int\n +- LocalTableScan [value#128]\n","sparkPlanInfo":{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [input[0, int, false] AS value#134]","children":[{"nodeName":"MapElements","simpleString":"MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2489/1993254639@5aaa636f, obj#133: int","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject value#128: int, obj#132: int","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"LocalTableScan","simpleString":"LocalTableScan [value#128]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":511,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration total (min, med, max)","accumulatorId":510,"metricType":"timing"}]},"time":1547788474349} -{"Event":"SparkListenerJobStart","Job ID":13,"Submission Time":1547788474403,"Stage Infos":[{"Stage ID":18,"Stage Attempt ID":0,"Stage Name":"collect at SparkSQLExample.scala:175","Number of Tasks":2,"RDD Info":[{"RDD ID":41,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"95\",\"name\":\"mapPartitionsInternal\"}","Callsite":"collect at SparkSQLExample.scala:175","Parent IDs":[40],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":38,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"94\",\"name\":\"LocalTableScan\"}","Callsite":"collect at SparkSQLExample.scala:175","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":39,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"94\",\"name\":\"LocalTableScan\"}","Callsite":"collect at SparkSQLExample.scala:175","Parent IDs":[38],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":40,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"89\",\"name\":\"WholeStageCodegen\"}","Callsite":"collect at SparkSQLExample.scala:175","Parent IDs":[39],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.collect(Dataset.scala:2708)\norg.apache.spark.examples.sql.SparkSQLExample$.runDatasetCreationExample(SparkSQLExample.scala:175)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:50)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[18],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.rdd.scope":"{\"id\":\"96\",\"name\":\"collect\"}","spark.rdd.scope.noOverride":"true","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"11","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":18,"Stage Attempt ID":0,"Stage Name":"collect at SparkSQLExample.scala:175","Number of Tasks":2,"RDD Info":[{"RDD ID":41,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"95\",\"name\":\"mapPartitionsInternal\"}","Callsite":"collect at SparkSQLExample.scala:175","Parent IDs":[40],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":38,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"94\",\"name\":\"LocalTableScan\"}","Callsite":"collect at SparkSQLExample.scala:175","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":39,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"94\",\"name\":\"LocalTableScan\"}","Callsite":"collect at SparkSQLExample.scala:175","Parent IDs":[38],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":40,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"89\",\"name\":\"WholeStageCodegen\"}","Callsite":"collect at SparkSQLExample.scala:175","Parent IDs":[39],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.collect(Dataset.scala:2708)\norg.apache.spark.examples.sql.SparkSQLExample$.runDatasetCreationExample(SparkSQLExample.scala:175)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:50)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788474405,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.rdd.scope":"{\"id\":\"96\",\"name\":\"collect\"}","spark.rdd.scope.noOverride":"true","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"11","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerTaskStart","Stage ID":18,"Stage Attempt ID":0,"Task Info":{"Task ID":209,"Index":0,"Attempt":0,"Launch Time":1547788474411,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":18,"Stage Attempt ID":0,"Task Info":{"Task ID":210,"Index":1,"Attempt":0,"Launch Time":1547788474414,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":18,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":210,"Index":1,"Attempt":0,"Launch Time":1547788474414,"Executor ID":"1","Host":"node3","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788474508,"Failed":false,"Killed":false,"Accumulables":[{"ID":511,"Name":"number of output rows","Update":"2","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":516,"Name":"internal.metrics.resultSize","Update":1266,"Value":1266,"Internal":true,"Count Failed Values":true},{"ID":515,"Name":"internal.metrics.executorCpuTime","Update":27108842,"Value":27108842,"Internal":true,"Count Failed Values":true},{"ID":514,"Name":"internal.metrics.executorRunTime","Update":31,"Value":31,"Internal":true,"Count Failed Values":true},{"ID":513,"Name":"internal.metrics.executorDeserializeCpuTime","Update":53942156,"Value":53942156,"Internal":true,"Count Failed Values":true},{"ID":512,"Name":"internal.metrics.executorDeserializeTime","Update":57,"Value":57,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":57,"Executor Deserialize CPU Time":53942156,"Executor Run Time":31,"Executor CPU Time":27108842,"Result Size":1266,"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":18,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":209,"Index":0,"Attempt":0,"Launch Time":1547788474411,"Executor ID":"2","Host":"node4","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788474549,"Failed":false,"Killed":false,"Accumulables":[{"ID":510,"Name":"duration total (min, med, max)","Update":"0","Value":"-2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":511,"Name":"number of output rows","Update":"1","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":518,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":516,"Name":"internal.metrics.resultSize","Update":1302,"Value":2568,"Internal":true,"Count Failed Values":true},{"ID":515,"Name":"internal.metrics.executorCpuTime","Update":36033484,"Value":63142326,"Internal":true,"Count Failed Values":true},{"ID":514,"Name":"internal.metrics.executorRunTime","Update":45,"Value":76,"Internal":true,"Count Failed Values":true},{"ID":513,"Name":"internal.metrics.executorDeserializeCpuTime","Update":51814579,"Value":105756735,"Internal":true,"Count Failed Values":true},{"ID":512,"Name":"internal.metrics.executorDeserializeTime","Update":80,"Value":137,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":80,"Executor Deserialize CPU Time":51814579,"Executor Run Time":45,"Executor CPU Time":36033484,"Result Size":1302,"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":"SparkListenerStageCompleted","Stage Info":{"Stage ID":18,"Stage Attempt ID":0,"Stage Name":"collect at SparkSQLExample.scala:175","Number of Tasks":2,"RDD Info":[{"RDD ID":41,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"95\",\"name\":\"mapPartitionsInternal\"}","Callsite":"collect at SparkSQLExample.scala:175","Parent IDs":[40],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":38,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"94\",\"name\":\"LocalTableScan\"}","Callsite":"collect at SparkSQLExample.scala:175","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":39,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"94\",\"name\":\"LocalTableScan\"}","Callsite":"collect at SparkSQLExample.scala:175","Parent IDs":[38],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":40,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"89\",\"name\":\"WholeStageCodegen\"}","Callsite":"collect at SparkSQLExample.scala:175","Parent IDs":[39],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.collect(Dataset.scala:2708)\norg.apache.spark.examples.sql.SparkSQLExample$.runDatasetCreationExample(SparkSQLExample.scala:175)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:50)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788474405,"Completion Time":1547788474550,"Accumulables":[{"ID":514,"Name":"internal.metrics.executorRunTime","Value":76,"Internal":true,"Count Failed Values":true},{"ID":516,"Name":"internal.metrics.resultSize","Value":2568,"Internal":true,"Count Failed Values":true},{"ID":510,"Name":"duration total (min, med, max)","Value":"-2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":513,"Name":"internal.metrics.executorDeserializeCpuTime","Value":105756735,"Internal":true,"Count Failed Values":true},{"ID":512,"Name":"internal.metrics.executorDeserializeTime","Value":137,"Internal":true,"Count Failed Values":true},{"ID":515,"Name":"internal.metrics.executorCpuTime","Value":63142326,"Internal":true,"Count Failed Values":true},{"ID":518,"Name":"internal.metrics.resultSerializationTime","Value":1,"Internal":true,"Count Failed Values":true},{"ID":511,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"}]}} -{"Event":"SparkListenerJobEnd","Job ID":13,"Completion Time":1547788474551,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":11,"time":1547788474555} -{"Event":"SparkListenerJobStart","Job ID":14,"Submission Time":1547788474640,"Stage Infos":[{"Stage ID":19,"Stage Attempt ID":0,"Stage Name":"json at SparkSQLExample.scala:179","Number of Tasks":1,"RDD Info":[{"RDD ID":44,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"103\",\"name\":\"mapPartitions\"}","Callsite":"json at SparkSQLExample.scala:179","Parent IDs":[43],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":42,"Name":"FileScanRDD","Scope":"{\"id\":\"101\",\"name\":\"WholeStageCodegen\"}","Callsite":"json at SparkSQLExample.scala:179","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":43,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"101\",\"name\":\"WholeStageCodegen\"}","Callsite":"json at SparkSQLExample.scala:179","Parent IDs":[42],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameReader.json(DataFrameReader.scala:331)\norg.apache.spark.examples.sql.SparkSQLExample$.runDatasetCreationExample(SparkSQLExample.scala:179)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:50)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[19],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":19,"Stage Attempt ID":0,"Stage Name":"json at SparkSQLExample.scala:179","Number of Tasks":1,"RDD Info":[{"RDD ID":44,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"103\",\"name\":\"mapPartitions\"}","Callsite":"json at SparkSQLExample.scala:179","Parent IDs":[43],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":42,"Name":"FileScanRDD","Scope":"{\"id\":\"101\",\"name\":\"WholeStageCodegen\"}","Callsite":"json at SparkSQLExample.scala:179","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":43,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"101\",\"name\":\"WholeStageCodegen\"}","Callsite":"json at SparkSQLExample.scala:179","Parent IDs":[42],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameReader.json(DataFrameReader.scala:331)\norg.apache.spark.examples.sql.SparkSQLExample$.runDatasetCreationExample(SparkSQLExample.scala:179)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:50)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788474641,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerTaskStart","Stage ID":19,"Stage Attempt ID":0,"Task Info":{"Task ID":211,"Index":0,"Attempt":0,"Launch Time":1547788474646,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":19,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":211,"Index":0,"Attempt":0,"Launch Time":1547788474646,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788474710,"Failed":false,"Killed":false,"Accumulables":[{"ID":541,"Name":"duration total (min, med, max)","Update":"41","Value":"40","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":537,"Name":"number of output rows","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":564,"Name":"internal.metrics.input.recordsRead","Update":3,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":563,"Name":"internal.metrics.input.bytesRead","Update":73,"Value":73,"Internal":true,"Count Failed Values":true},{"ID":546,"Name":"internal.metrics.resultSize","Update":1877,"Value":1877,"Internal":true,"Count Failed Values":true},{"ID":545,"Name":"internal.metrics.executorCpuTime","Update":36440214,"Value":36440214,"Internal":true,"Count Failed Values":true},{"ID":544,"Name":"internal.metrics.executorRunTime","Update":45,"Value":45,"Internal":true,"Count Failed Values":true},{"ID":543,"Name":"internal.metrics.executorDeserializeCpuTime","Update":9970752,"Value":9970752,"Internal":true,"Count Failed Values":true},{"ID":542,"Name":"internal.metrics.executorDeserializeTime","Update":14,"Value":14,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":14,"Executor Deserialize CPU Time":9970752,"Executor Run Time":45,"Executor CPU Time":36440214,"Result Size":1877,"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":73,"Records Read":3},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":19,"Stage Attempt ID":0,"Stage Name":"json at SparkSQLExample.scala:179","Number of Tasks":1,"RDD Info":[{"RDD ID":44,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"103\",\"name\":\"mapPartitions\"}","Callsite":"json at SparkSQLExample.scala:179","Parent IDs":[43],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":42,"Name":"FileScanRDD","Scope":"{\"id\":\"101\",\"name\":\"WholeStageCodegen\"}","Callsite":"json at SparkSQLExample.scala:179","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":43,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"101\",\"name\":\"WholeStageCodegen\"}","Callsite":"json at SparkSQLExample.scala:179","Parent IDs":[42],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameReader.json(DataFrameReader.scala:331)\norg.apache.spark.examples.sql.SparkSQLExample$.runDatasetCreationExample(SparkSQLExample.scala:179)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:50)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788474641,"Completion Time":1547788474711,"Accumulables":[{"ID":541,"Name":"duration total (min, med, max)","Value":"40","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":543,"Name":"internal.metrics.executorDeserializeCpuTime","Value":9970752,"Internal":true,"Count Failed Values":true},{"ID":537,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":564,"Name":"internal.metrics.input.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":546,"Name":"internal.metrics.resultSize","Value":1877,"Internal":true,"Count Failed Values":true},{"ID":563,"Name":"internal.metrics.input.bytesRead","Value":73,"Internal":true,"Count Failed Values":true},{"ID":542,"Name":"internal.metrics.executorDeserializeTime","Value":14,"Internal":true,"Count Failed Values":true},{"ID":545,"Name":"internal.metrics.executorCpuTime","Value":36440214,"Internal":true,"Count Failed Values":true},{"ID":544,"Name":"internal.metrics.executorRunTime","Value":45,"Internal":true,"Count Failed Values":true}]}} -{"Event":"SparkListenerJobEnd","Job ID":14,"Completion Time":1547788474711,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":12,"description":"show at SparkSQLExample.scala:180","details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runDatasetCreationExample(SparkSQLExample.scala:180)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:50)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#144L as string) AS age#155, cast(name#145 as string) AS name#156]\n +- Relation[age#144L,name#145] json\n\n== Analyzed Logical Plan ==\nage: string, name: string\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#144L as string) AS age#155, cast(name#145 as string) AS name#156]\n +- Relation[age#144L,name#145] json\n\n== Optimized Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(age#144L as string) AS age#155, name#145]\n +- Relation[age#144L,name#145] json\n\n== Physical Plan ==\nCollectLimit 21\n+- *(1) Project [cast(age#144L as string) AS age#155, name#145]\n +- *(1) FileScan json [age#144L,name#145] Batched: false, DataFilters: [], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct\n","sparkPlanInfo":{"nodeName":"CollectLimit","simpleString":"CollectLimit 21","children":[{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"Project","simpleString":"Project [cast(age#144L as string) AS age#155, name#145]","children":[{"nodeName":"Scan json ","simpleString":"FileScan json [age#144L,name#145] Batched: false, DataFilters: [], Format: JSON, Location: InMemoryFileIndex[hdfs://node6:8020/user/spark/examples..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct","children":[],"metadata":{"Location":"InMemoryFileIndex[hdfs://node6:8020/user/spark/examples/src/main/resources/people.json]","ReadSchema":"struct","Format":"JSON","Batched":"false","PartitionFilters":"[]","PushedFilters":"[]","DataFilters":"[]"},"metrics":[{"name":"number of output rows","accumulatorId":578,"metricType":"sum"},{"name":"number of files","accumulatorId":579,"metricType":"sum"},{"name":"metadata time","accumulatorId":580,"metricType":"sum"},{"name":"scan time total (min, med, max)","accumulatorId":581,"metricType":"timing"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration total (min, med, max)","accumulatorId":577,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":575,"metricType":"sum"},{"name":"shuffle write time total (min, med, max)","accumulatorId":576,"metricType":"nsTiming"},{"name":"records read","accumulatorId":573,"metricType":"sum"},{"name":"local bytes read total (min, med, max)","accumulatorId":571,"metricType":"size"},{"name":"fetch wait time total (min, med, max)","accumulatorId":572,"metricType":"timing"},{"name":"remote bytes read total (min, med, max)","accumulatorId":569,"metricType":"size"},{"name":"local blocks read","accumulatorId":568,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":567,"metricType":"sum"},{"name":"remote bytes read to disk total (min, med, max)","accumulatorId":570,"metricType":"size"},{"name":"shuffle bytes written total (min, med, max)","accumulatorId":574,"metricType":"size"}]},"time":1547788474761} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates","executionId":12,"accumUpdates":[[579,1],[580,0]]} -{"Event":"SparkListenerJobStart","Job ID":15,"Submission Time":1547788474797,"Stage Infos":[{"Stage ID":20,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:180","Number of Tasks":1,"RDD Info":[{"RDD ID":48,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"111\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:180","Parent IDs":[47],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":45,"Name":"FileScanRDD","Scope":"{\"id\":\"107\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:180","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":46,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"107\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:180","Parent IDs":[45],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":47,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"110\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:180","Parent IDs":[46],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runDatasetCreationExample(SparkSQLExample.scala:180)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:50)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[20],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"12","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":20,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:180","Number of Tasks":1,"RDD Info":[{"RDD ID":48,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"111\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:180","Parent IDs":[47],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":45,"Name":"FileScanRDD","Scope":"{\"id\":\"107\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:180","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":46,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"107\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:180","Parent IDs":[45],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":47,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"110\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:180","Parent IDs":[46],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runDatasetCreationExample(SparkSQLExample.scala:180)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:50)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788474798,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"12","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerTaskStart","Stage ID":20,"Stage Attempt ID":0,"Task Info":{"Task ID":212,"Index":0,"Attempt":0,"Launch Time":1547788474804,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":20,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":212,"Index":0,"Attempt":0,"Launch Time":1547788474804,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788474869,"Failed":false,"Killed":false,"Accumulables":[{"ID":577,"Name":"duration total (min, med, max)","Update":"44","Value":"43","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":578,"Name":"number of output rows","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":604,"Name":"internal.metrics.input.recordsRead","Update":3,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":603,"Name":"internal.metrics.input.bytesRead","Update":73,"Value":73,"Internal":true,"Count Failed Values":true},{"ID":586,"Name":"internal.metrics.resultSize","Update":1229,"Value":1229,"Internal":true,"Count Failed Values":true},{"ID":585,"Name":"internal.metrics.executorCpuTime","Update":39247717,"Value":39247717,"Internal":true,"Count Failed Values":true},{"ID":584,"Name":"internal.metrics.executorRunTime","Update":46,"Value":46,"Internal":true,"Count Failed Values":true},{"ID":583,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8752334,"Value":8752334,"Internal":true,"Count Failed Values":true},{"ID":582,"Name":"internal.metrics.executorDeserializeTime","Update":13,"Value":13,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":13,"Executor Deserialize CPU Time":8752334,"Executor Run Time":46,"Executor CPU Time":39247717,"Result Size":1229,"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":73,"Records Read":3},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":20,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:180","Number of Tasks":1,"RDD Info":[{"RDD ID":48,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"111\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:180","Parent IDs":[47],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":45,"Name":"FileScanRDD","Scope":"{\"id\":\"107\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:180","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":46,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"107\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:180","Parent IDs":[45],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":47,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"110\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:180","Parent IDs":[46],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runDatasetCreationExample(SparkSQLExample.scala:180)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:50)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788474798,"Completion Time":1547788474870,"Accumulables":[{"ID":577,"Name":"duration total (min, med, max)","Value":"43","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":603,"Name":"internal.metrics.input.bytesRead","Value":73,"Internal":true,"Count Failed Values":true},{"ID":585,"Name":"internal.metrics.executorCpuTime","Value":39247717,"Internal":true,"Count Failed Values":true},{"ID":582,"Name":"internal.metrics.executorDeserializeTime","Value":13,"Internal":true,"Count Failed Values":true},{"ID":584,"Name":"internal.metrics.executorRunTime","Value":46,"Internal":true,"Count Failed Values":true},{"ID":578,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":604,"Name":"internal.metrics.input.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":586,"Name":"internal.metrics.resultSize","Value":1229,"Internal":true,"Count Failed Values":true},{"ID":583,"Name":"internal.metrics.executorDeserializeCpuTime","Value":8752334,"Internal":true,"Count Failed Values":true}]}} -{"Event":"SparkListenerJobEnd","Job ID":15,"Completion Time":1547788474870,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":12,"time":1547788474871} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":13,"description":"createOrReplaceTempView at SparkSQLExample.scala:203","details":"org.apache.spark.sql.Dataset.createOrReplaceTempView(Dataset.scala:3021)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:203)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\nCreateViewCommand `people`, false, true, LocalTempView\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).name, true, false) AS name#167, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).age AS age#168L]\n +- ExternalRDD [obj#166]\n\n== Analyzed Logical Plan ==\n\nCreateViewCommand `people`, false, true, LocalTempView\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).name, true, false) AS name#167, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).age AS age#168L]\n +- ExternalRDD [obj#166]\n\n== Optimized Logical Plan ==\nCreateViewCommand `people`, false, true, LocalTempView\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).name, true, false) AS name#167, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).age AS age#168L]\n +- ExternalRDD [obj#166]\n\n== Physical Plan ==\nExecute CreateViewCommand\n +- CreateViewCommand `people`, false, true, LocalTempView\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).name, true, false) AS name#167, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).age AS age#168L]\n +- ExternalRDD [obj#166]\n","sparkPlanInfo":{"nodeName":"Execute CreateViewCommand","simpleString":"Execute CreateViewCommand","children":[],"metadata":{},"metrics":[]},"time":1547788474960} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":13,"time":1547788474960} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":14,"description":"show at SparkSQLExample.scala:209","details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:209)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(value#179 as string) AS value#183]\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#179]\n +- MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2518/869554567@d33a96c, interface org.apache.spark.sql.Row, [StructField(name,StringType,true), StructField(age,LongType,false)], obj#178: java.lang.String\n +- DeserializeToObject createexternalrow(name#167.toString, age#168L, StructField(name,StringType,true), StructField(age,LongType,false)), obj#177: org.apache.spark.sql.Row\n +- Project [name#167, age#168L]\n +- Filter ((age#168L >= cast(13 as bigint)) && (age#168L <= cast(19 as bigint)))\n +- SubqueryAlias `people`\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).name, true, false) AS name#167, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).age AS age#168L]\n +- ExternalRDD [obj#166]\n\n== Analyzed Logical Plan ==\nvalue: string\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(value#179 as string) AS value#183]\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#179]\n +- MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2518/869554567@d33a96c, interface org.apache.spark.sql.Row, [StructField(name,StringType,true), StructField(age,LongType,false)], obj#178: java.lang.String\n +- DeserializeToObject createexternalrow(name#167.toString, age#168L, StructField(name,StringType,true), StructField(age,LongType,false)), obj#177: org.apache.spark.sql.Row\n +- Project [name#167, age#168L]\n +- Filter ((age#168L >= cast(13 as bigint)) && (age#168L <= cast(19 as bigint)))\n +- SubqueryAlias `people`\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).name, true, false) AS name#167, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).age AS age#168L]\n +- ExternalRDD [obj#166]\n\n== Optimized Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#179]\n +- MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2518/869554567@d33a96c, interface org.apache.spark.sql.Row, [StructField(name,StringType,true), StructField(age,LongType,false)], obj#178: java.lang.String\n +- DeserializeToObject createexternalrow(name#167.toString, age#168L, StructField(name,StringType,true), StructField(age,LongType,false)), obj#177: org.apache.spark.sql.Row\n +- Filter ((age#168L >= 13) && (age#168L <= 19))\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).name, true, false) AS name#167, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).age AS age#168L]\n +- ExternalRDD [obj#166]\n\n== Physical Plan ==\nCollectLimit 21\n+- *(1) SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#179]\n +- *(1) MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2518/869554567@d33a96c, obj#178: java.lang.String\n +- *(1) DeserializeToObject createexternalrow(name#167.toString, age#168L, StructField(name,StringType,true), StructField(age,LongType,false)), obj#177: org.apache.spark.sql.Row\n +- *(1) Filter ((age#168L >= 13) && (age#168L <= 19))\n +- *(1) SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).name, true, false) AS name#167, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).age AS age#168L]\n +- Scan[obj#166]\n","sparkPlanInfo":{"nodeName":"CollectLimit","simpleString":"CollectLimit 21","children":[{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#179]","children":[{"nodeName":"MapElements","simpleString":"MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2518/869554567@d33a96c, obj#178: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject createexternalrow(name#167.toString, age#168L, StructField(name,StringType,true), StructField(age,LongType,false)), obj#177: org.apache.spark.sql.Row","children":[{"nodeName":"Filter","simpleString":"Filter ((age#168L >= 13) && (age#168L <= 19))","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).name, true, false) AS name#167, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).age AS age#168L]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Scan","simpleString":"Scan[obj#166]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":619,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":618,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration total (min, med, max)","accumulatorId":617,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":615,"metricType":"sum"},{"name":"shuffle write time total (min, med, max)","accumulatorId":616,"metricType":"nsTiming"},{"name":"records read","accumulatorId":613,"metricType":"sum"},{"name":"local bytes read total (min, med, max)","accumulatorId":611,"metricType":"size"},{"name":"fetch wait time total (min, med, max)","accumulatorId":612,"metricType":"timing"},{"name":"remote bytes read total (min, med, max)","accumulatorId":609,"metricType":"size"},{"name":"local blocks read","accumulatorId":608,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":607,"metricType":"sum"},{"name":"remote bytes read to disk total (min, med, max)","accumulatorId":610,"metricType":"size"},{"name":"shuffle bytes written total (min, med, max)","accumulatorId":614,"metricType":"size"}]},"time":1547788475059} -{"Event":"SparkListenerJobStart","Job ID":16,"Submission Time":1547788475137,"Stage Infos":[{"Stage ID":21,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:209","Number of Tasks":1,"RDD Info":[{"RDD ID":56,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"128\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[55],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"118\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:200","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":53,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"126\",\"name\":\"Scan\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":55,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"127\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[54],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":54,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"119\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[53],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"117\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:199","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:209)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[21],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"14","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":21,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:209","Number of Tasks":1,"RDD Info":[{"RDD ID":56,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"128\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[55],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"118\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:200","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":53,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"126\",\"name\":\"Scan\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":55,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"127\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[54],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":54,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"119\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[53],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"117\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:199","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:209)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788475140,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"14","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerTaskStart","Stage ID":21,"Stage Attempt ID":0,"Task Info":{"Task ID":213,"Index":0,"Attempt":0,"Launch Time":1547788475183,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":21,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":213,"Index":0,"Attempt":0,"Launch Time":1547788475183,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788475422,"Failed":false,"Killed":false,"Accumulables":[{"ID":617,"Name":"duration total (min, med, max)","Update":"3","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":619,"Name":"number of output rows","Update":"2","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":642,"Name":"internal.metrics.input.recordsRead","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":641,"Name":"internal.metrics.input.bytesRead","Update":32,"Value":32,"Internal":true,"Count Failed Values":true},{"ID":624,"Name":"internal.metrics.resultSize","Update":1208,"Value":1208,"Internal":true,"Count Failed Values":true},{"ID":623,"Name":"internal.metrics.executorCpuTime","Update":106404621,"Value":106404621,"Internal":true,"Count Failed Values":true},{"ID":622,"Name":"internal.metrics.executorRunTime","Update":123,"Value":123,"Internal":true,"Count Failed Values":true},{"ID":621,"Name":"internal.metrics.executorDeserializeCpuTime","Update":46663283,"Value":46663283,"Internal":true,"Count Failed Values":true},{"ID":620,"Name":"internal.metrics.executorDeserializeTime","Update":100,"Value":100,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":100,"Executor Deserialize CPU Time":46663283,"Executor Run Time":123,"Executor CPU Time":106404621,"Result Size":1208,"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":32,"Records Read":2},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":21,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:209","Number of Tasks":1,"RDD Info":[{"RDD ID":56,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"128\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[55],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"118\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:200","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":53,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"126\",\"name\":\"Scan\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":55,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"127\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[54],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":54,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"119\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[53],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"117\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:199","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:209)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788475140,"Completion Time":1547788475423,"Accumulables":[{"ID":621,"Name":"internal.metrics.executorDeserializeCpuTime","Value":46663283,"Internal":true,"Count Failed Values":true},{"ID":642,"Name":"internal.metrics.input.recordsRead","Value":2,"Internal":true,"Count Failed Values":true},{"ID":624,"Name":"internal.metrics.resultSize","Value":1208,"Internal":true,"Count Failed Values":true},{"ID":641,"Name":"internal.metrics.input.bytesRead","Value":32,"Internal":true,"Count Failed Values":true},{"ID":623,"Name":"internal.metrics.executorCpuTime","Value":106404621,"Internal":true,"Count Failed Values":true},{"ID":617,"Name":"duration total (min, med, max)","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":620,"Name":"internal.metrics.executorDeserializeTime","Value":100,"Internal":true,"Count Failed Values":true},{"ID":622,"Name":"internal.metrics.executorRunTime","Value":123,"Internal":true,"Count Failed Values":true},{"ID":619,"Name":"number of output rows","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"}]}} -{"Event":"SparkListenerJobEnd","Job ID":16,"Completion Time":1547788475423,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerJobStart","Job ID":17,"Submission Time":1547788475432,"Stage Infos":[{"Stage ID":22,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:209","Number of Tasks":1,"RDD Info":[{"RDD ID":56,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"128\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[55],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"118\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:200","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":53,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"126\",\"name\":\"Scan\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":55,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"127\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[54],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":54,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"119\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[53],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"117\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:199","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:209)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[22],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"14","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":22,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:209","Number of Tasks":1,"RDD Info":[{"RDD ID":56,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"128\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[55],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"118\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:200","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":53,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"126\",\"name\":\"Scan\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":55,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"127\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[54],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":54,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"119\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[53],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"117\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:199","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:209)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788475433,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"14","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerTaskStart","Stage ID":22,"Stage Attempt ID":0,"Task Info":{"Task ID":214,"Index":0,"Attempt":0,"Launch Time":1547788475440,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":22,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":214,"Index":0,"Attempt":0,"Launch Time":1547788475440,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788475466,"Failed":false,"Killed":false,"Accumulables":[{"ID":618,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":619,"Name":"number of output rows","Update":"1","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":667,"Name":"internal.metrics.input.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":666,"Name":"internal.metrics.input.bytesRead","Update":16,"Value":16,"Internal":true,"Count Failed Values":true},{"ID":649,"Name":"internal.metrics.resultSize","Update":1242,"Value":1242,"Internal":true,"Count Failed Values":true},{"ID":648,"Name":"internal.metrics.executorCpuTime","Update":7188516,"Value":7188516,"Internal":true,"Count Failed Values":true},{"ID":647,"Name":"internal.metrics.executorRunTime","Update":10,"Value":10,"Internal":true,"Count Failed Values":true},{"ID":646,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7299108,"Value":7299108,"Internal":true,"Count Failed Values":true},{"ID":645,"Name":"internal.metrics.executorDeserializeTime","Update":11,"Value":11,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":11,"Executor Deserialize CPU Time":7299108,"Executor Run Time":10,"Executor CPU Time":7188516,"Result Size":1242,"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":16,"Records Read":1},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":22,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:209","Number of Tasks":1,"RDD Info":[{"RDD ID":56,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"128\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[55],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"118\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:200","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":53,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"126\",\"name\":\"Scan\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":55,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"127\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[54],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":54,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"119\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:209","Parent IDs":[53],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"117\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:199","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:209)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788475433,"Completion Time":1547788475467,"Accumulables":[{"ID":618,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":645,"Name":"internal.metrics.executorDeserializeTime","Value":11,"Internal":true,"Count Failed Values":true},{"ID":666,"Name":"internal.metrics.input.bytesRead","Value":16,"Internal":true,"Count Failed Values":true},{"ID":648,"Name":"internal.metrics.executorCpuTime","Value":7188516,"Internal":true,"Count Failed Values":true},{"ID":647,"Name":"internal.metrics.executorRunTime","Value":10,"Internal":true,"Count Failed Values":true},{"ID":646,"Name":"internal.metrics.executorDeserializeCpuTime","Value":7299108,"Internal":true,"Count Failed Values":true},{"ID":667,"Name":"internal.metrics.input.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":649,"Name":"internal.metrics.resultSize","Value":1242,"Internal":true,"Count Failed Values":true},{"ID":619,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"}]}} -{"Event":"SparkListenerJobEnd","Job ID":17,"Completion Time":1547788475467,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":14,"time":1547788475468} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":15,"description":"show at SparkSQLExample.scala:217","details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:217)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(value#190 as string) AS value#194]\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#190]\n +- MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2541/1946069198@a0fc978, interface org.apache.spark.sql.Row, [StructField(name,StringType,true), StructField(age,LongType,false)], obj#189: java.lang.String\n +- DeserializeToObject createexternalrow(name#167.toString, age#168L, StructField(name,StringType,true), StructField(age,LongType,false)), obj#188: org.apache.spark.sql.Row\n +- Project [name#167, age#168L]\n +- Filter ((age#168L >= cast(13 as bigint)) && (age#168L <= cast(19 as bigint)))\n +- SubqueryAlias `people`\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).name, true, false) AS name#167, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).age AS age#168L]\n +- ExternalRDD [obj#166]\n\n== Analyzed Logical Plan ==\nvalue: string\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(value#190 as string) AS value#194]\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#190]\n +- MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2541/1946069198@a0fc978, interface org.apache.spark.sql.Row, [StructField(name,StringType,true), StructField(age,LongType,false)], obj#189: java.lang.String\n +- DeserializeToObject createexternalrow(name#167.toString, age#168L, StructField(name,StringType,true), StructField(age,LongType,false)), obj#188: org.apache.spark.sql.Row\n +- Project [name#167, age#168L]\n +- Filter ((age#168L >= cast(13 as bigint)) && (age#168L <= cast(19 as bigint)))\n +- SubqueryAlias `people`\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).name, true, false) AS name#167, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).age AS age#168L]\n +- ExternalRDD [obj#166]\n\n== Optimized Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#190]\n +- MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2541/1946069198@a0fc978, interface org.apache.spark.sql.Row, [StructField(name,StringType,true), StructField(age,LongType,false)], obj#189: java.lang.String\n +- DeserializeToObject createexternalrow(name#167.toString, age#168L, StructField(name,StringType,true), StructField(age,LongType,false)), obj#188: org.apache.spark.sql.Row\n +- Filter ((age#168L >= 13) && (age#168L <= 19))\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).name, true, false) AS name#167, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).age AS age#168L]\n +- ExternalRDD [obj#166]\n\n== Physical Plan ==\nCollectLimit 21\n+- *(1) SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#190]\n +- *(1) MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2541/1946069198@a0fc978, obj#189: java.lang.String\n +- *(1) DeserializeToObject createexternalrow(name#167.toString, age#168L, StructField(name,StringType,true), StructField(age,LongType,false)), obj#188: org.apache.spark.sql.Row\n +- *(1) Filter ((age#168L >= 13) && (age#168L <= 19))\n +- *(1) SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).name, true, false) AS name#167, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).age AS age#168L]\n +- Scan[obj#166]\n","sparkPlanInfo":{"nodeName":"CollectLimit","simpleString":"CollectLimit 21","children":[{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#190]","children":[{"nodeName":"MapElements","simpleString":"MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2541/1946069198@a0fc978, obj#189: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject createexternalrow(name#167.toString, age#168L, StructField(name,StringType,true), StructField(age,LongType,false)), obj#188: org.apache.spark.sql.Row","children":[{"nodeName":"Filter","simpleString":"Filter ((age#168L >= 13) && (age#168L <= 19))","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).name, true, false) AS name#167, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).age AS age#168L]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Scan","simpleString":"Scan[obj#166]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":682,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":681,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration total (min, med, max)","accumulatorId":680,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":678,"metricType":"sum"},{"name":"shuffle write time total (min, med, max)","accumulatorId":679,"metricType":"nsTiming"},{"name":"records read","accumulatorId":676,"metricType":"sum"},{"name":"local bytes read total (min, med, max)","accumulatorId":674,"metricType":"size"},{"name":"fetch wait time total (min, med, max)","accumulatorId":675,"metricType":"timing"},{"name":"remote bytes read total (min, med, max)","accumulatorId":672,"metricType":"size"},{"name":"local blocks read","accumulatorId":671,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":670,"metricType":"sum"},{"name":"remote bytes read to disk total (min, med, max)","accumulatorId":673,"metricType":"size"},{"name":"shuffle bytes written total (min, med, max)","accumulatorId":677,"metricType":"size"}]},"time":1547788475520} -{"Event":"SparkListenerJobStart","Job ID":18,"Submission Time":1547788475541,"Stage Infos":[{"Stage ID":23,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:217","Number of Tasks":1,"RDD Info":[{"RDD ID":60,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"147\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[59],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"118\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:200","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":59,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"146\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[58],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":57,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"145\",\"name\":\"Scan\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":58,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"138\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[57],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"117\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:199","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:217)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[23],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"15","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":23,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:217","Number of Tasks":1,"RDD Info":[{"RDD ID":60,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"147\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[59],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"118\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:200","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":59,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"146\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[58],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":57,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"145\",\"name\":\"Scan\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":58,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"138\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[57],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"117\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:199","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:217)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788475542,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"15","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerTaskStart","Stage ID":23,"Stage Attempt ID":0,"Task Info":{"Task ID":215,"Index":0,"Attempt":0,"Launch Time":1547788475549,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":23,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":215,"Index":0,"Attempt":0,"Launch Time":1547788475549,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788475580,"Failed":false,"Killed":false,"Accumulables":[{"ID":680,"Name":"duration total (min, med, max)","Update":"2","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":682,"Name":"number of output rows","Update":"2","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":705,"Name":"internal.metrics.input.recordsRead","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":704,"Name":"internal.metrics.input.bytesRead","Update":32,"Value":32,"Internal":true,"Count Failed Values":true},{"ID":687,"Name":"internal.metrics.resultSize","Update":1208,"Value":1208,"Internal":true,"Count Failed Values":true},{"ID":686,"Name":"internal.metrics.executorCpuTime","Update":7352607,"Value":7352607,"Internal":true,"Count Failed Values":true},{"ID":685,"Name":"internal.metrics.executorRunTime","Update":9,"Value":9,"Internal":true,"Count Failed Values":true},{"ID":684,"Name":"internal.metrics.executorDeserializeCpuTime","Update":9475942,"Value":9475942,"Internal":true,"Count Failed Values":true},{"ID":683,"Name":"internal.metrics.executorDeserializeTime","Update":16,"Value":16,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":16,"Executor Deserialize CPU Time":9475942,"Executor Run Time":9,"Executor CPU Time":7352607,"Result Size":1208,"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":32,"Records Read":2},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":23,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:217","Number of Tasks":1,"RDD Info":[{"RDD ID":60,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"147\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[59],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"118\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:200","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":59,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"146\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[58],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":57,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"145\",\"name\":\"Scan\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":58,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"138\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[57],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"117\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:199","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:217)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788475542,"Completion Time":1547788475580,"Accumulables":[{"ID":705,"Name":"internal.metrics.input.recordsRead","Value":2,"Internal":true,"Count Failed Values":true},{"ID":687,"Name":"internal.metrics.resultSize","Value":1208,"Internal":true,"Count Failed Values":true},{"ID":684,"Name":"internal.metrics.executorDeserializeCpuTime","Value":9475942,"Internal":true,"Count Failed Values":true},{"ID":683,"Name":"internal.metrics.executorDeserializeTime","Value":16,"Internal":true,"Count Failed Values":true},{"ID":704,"Name":"internal.metrics.input.bytesRead","Value":32,"Internal":true,"Count Failed Values":true},{"ID":686,"Name":"internal.metrics.executorCpuTime","Value":7352607,"Internal":true,"Count Failed Values":true},{"ID":680,"Name":"duration total (min, med, max)","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":682,"Name":"number of output rows","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":685,"Name":"internal.metrics.executorRunTime","Value":9,"Internal":true,"Count Failed Values":true}]}} -{"Event":"SparkListenerJobEnd","Job ID":18,"Completion Time":1547788475581,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerJobStart","Job ID":19,"Submission Time":1547788475593,"Stage Infos":[{"Stage ID":24,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:217","Number of Tasks":1,"RDD Info":[{"RDD ID":60,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"147\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[59],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"118\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:200","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":59,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"146\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[58],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":57,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"145\",\"name\":\"Scan\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":58,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"138\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[57],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"117\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:199","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:217)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[24],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"15","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":24,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:217","Number of Tasks":1,"RDD Info":[{"RDD ID":60,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"147\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[59],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"118\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:200","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":59,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"146\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[58],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":57,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"145\",\"name\":\"Scan\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":58,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"138\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[57],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"117\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:199","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:217)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788475594,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"15","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerTaskStart","Stage ID":24,"Stage Attempt ID":0,"Task Info":{"Task ID":216,"Index":0,"Attempt":0,"Launch Time":1547788475600,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":24,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":216,"Index":0,"Attempt":0,"Launch Time":1547788475600,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788475629,"Failed":false,"Killed":false,"Accumulables":[{"ID":681,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":682,"Name":"number of output rows","Update":"1","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":730,"Name":"internal.metrics.input.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":729,"Name":"internal.metrics.input.bytesRead","Update":16,"Value":16,"Internal":true,"Count Failed Values":true},{"ID":712,"Name":"internal.metrics.resultSize","Update":1242,"Value":1242,"Internal":true,"Count Failed Values":true},{"ID":711,"Name":"internal.metrics.executorCpuTime","Update":6828175,"Value":6828175,"Internal":true,"Count Failed Values":true},{"ID":710,"Name":"internal.metrics.executorRunTime","Update":9,"Value":9,"Internal":true,"Count Failed Values":true},{"ID":709,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8623587,"Value":8623587,"Internal":true,"Count Failed Values":true},{"ID":708,"Name":"internal.metrics.executorDeserializeTime","Update":14,"Value":14,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":14,"Executor Deserialize CPU Time":8623587,"Executor Run Time":9,"Executor CPU Time":6828175,"Result Size":1242,"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":16,"Records Read":1},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":24,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:217","Number of Tasks":1,"RDD Info":[{"RDD ID":60,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"147\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[59],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"118\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:200","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":59,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"146\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[58],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":57,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"145\",\"name\":\"Scan\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":58,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"138\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:217","Parent IDs":[57],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"117\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:199","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:217)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788475594,"Completion Time":1547788475630,"Accumulables":[{"ID":729,"Name":"internal.metrics.input.bytesRead","Value":16,"Internal":true,"Count Failed Values":true},{"ID":681,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":708,"Name":"internal.metrics.executorDeserializeTime","Value":14,"Internal":true,"Count Failed Values":true},{"ID":711,"Name":"internal.metrics.executorCpuTime","Value":6828175,"Internal":true,"Count Failed Values":true},{"ID":710,"Name":"internal.metrics.executorRunTime","Value":9,"Internal":true,"Count Failed Values":true},{"ID":682,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":709,"Name":"internal.metrics.executorDeserializeCpuTime","Value":8623587,"Internal":true,"Count Failed Values":true},{"ID":730,"Name":"internal.metrics.input.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":712,"Name":"internal.metrics.resultSize","Value":1242,"Internal":true,"Count Failed Values":true}]}} -{"Event":"SparkListenerJobEnd","Job ID":19,"Completion Time":1547788475630,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":15,"time":1547788475631} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":16,"description":"collect at SparkSQLExample.scala:230","details":"org.apache.spark.sql.Dataset.collect(Dataset.scala:2708)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:230)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\n'SerializeFromObject [encodeusingserializer(input[0, java.lang.Object, true], true) AS value#201]\n+- 'MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2542/1591023561@7dd320, interface org.apache.spark.sql.Row, [StructField(name,StringType,true), StructField(age,LongType,false)], obj#200: scala.collection.immutable.Map\n +- 'DeserializeToObject unresolveddeserializer(createexternalrow(getcolumnbyordinal(0, StructField(name,StringType,true), StructField(age,LongType,false)).toString, getcolumnbyordinal(1, StructField(name,StringType,true), StructField(age,LongType,false)), StructField(name,StringType,true), StructField(age,LongType,false))), obj#199: org.apache.spark.sql.Row\n +- Project [name#167, age#168L]\n +- Filter ((age#168L >= cast(13 as bigint)) && (age#168L <= cast(19 as bigint)))\n +- SubqueryAlias `people`\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).name, true, false) AS name#167, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).age AS age#168L]\n +- ExternalRDD [obj#166]\n\n== Analyzed Logical Plan ==\nvalue: binary\nSerializeFromObject [encodeusingserializer(input[0, java.lang.Object, true], true) AS value#201]\n+- MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2542/1591023561@7dd320, interface org.apache.spark.sql.Row, [StructField(name,StringType,true), StructField(age,LongType,false)], obj#200: scala.collection.immutable.Map\n +- DeserializeToObject createexternalrow(name#167.toString, age#168L, StructField(name,StringType,true), StructField(age,LongType,false)), obj#199: org.apache.spark.sql.Row\n +- Project [name#167, age#168L]\n +- Filter ((age#168L >= cast(13 as bigint)) && (age#168L <= cast(19 as bigint)))\n +- SubqueryAlias `people`\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).name, true, false) AS name#167, assertnotnull(assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true])).age AS age#168L]\n +- ExternalRDD [obj#166]\n\n== Optimized Logical Plan ==\nSerializeFromObject [encodeusingserializer(input[0, java.lang.Object, true], true) AS value#201]\n+- MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2542/1591023561@7dd320, interface org.apache.spark.sql.Row, [StructField(name,StringType,true), StructField(age,LongType,false)], obj#200: scala.collection.immutable.Map\n +- DeserializeToObject createexternalrow(name#167.toString, age#168L, StructField(name,StringType,true), StructField(age,LongType,false)), obj#199: org.apache.spark.sql.Row\n +- Filter ((age#168L >= 13) && (age#168L <= 19))\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).name, true, false) AS name#167, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).age AS age#168L]\n +- ExternalRDD [obj#166]\n\n== Physical Plan ==\n*(1) SerializeFromObject [encodeusingserializer(input[0, java.lang.Object, true], true) AS value#201]\n+- *(1) MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2542/1591023561@7dd320, obj#200: scala.collection.immutable.Map\n +- *(1) DeserializeToObject createexternalrow(name#167.toString, age#168L, StructField(name,StringType,true), StructField(age,LongType,false)), obj#199: org.apache.spark.sql.Row\n +- *(1) Filter ((age#168L >= 13) && (age#168L <= 19))\n +- *(1) SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).name, true, false) AS name#167, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).age AS age#168L]\n +- Scan[obj#166]\n","sparkPlanInfo":{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [encodeusingserializer(input[0, java.lang.Object, true], true) AS value#201]","children":[{"nodeName":"MapElements","simpleString":"MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2542/1591023561@7dd320, obj#200: scala.collection.immutable.Map","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject createexternalrow(name#167.toString, age#168L, StructField(name,StringType,true), StructField(age,LongType,false)), obj#199: org.apache.spark.sql.Row","children":[{"nodeName":"Filter","simpleString":"Filter ((age#168L >= 13) && (age#168L <= 19))","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).name, true, false) AS name#167, assertnotnull(input[0, org.apache.spark.examples.sql.SparkSQLExample$Person, true]).age AS age#168L]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"Scan","simpleString":"Scan[obj#166]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":735,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":734,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration total (min, med, max)","accumulatorId":733,"metricType":"timing"}]},"time":1547788475673} -{"Event":"SparkListenerJobStart","Job ID":20,"Submission Time":1547788475754,"Stage Infos":[{"Stage ID":25,"Stage Attempt ID":0,"Stage Name":"collect at SparkSQLExample.scala:230","Number of Tasks":2,"RDD Info":[{"RDD ID":63,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"162\",\"name\":\"mapPartitionsInternal\"}","Callsite":"collect at SparkSQLExample.scala:230","Parent IDs":[62],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"118\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:200","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":62,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"154\",\"name\":\"WholeStageCodegen\"}","Callsite":"collect at SparkSQLExample.scala:230","Parent IDs":[61],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":61,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"161\",\"name\":\"Scan\"}","Callsite":"collect at SparkSQLExample.scala:230","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"117\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:199","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.collect(Dataset.scala:2708)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:230)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[25],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.rdd.scope":"{\"id\":\"163\",\"name\":\"collect\"}","spark.rdd.scope.noOverride":"true","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"16","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":25,"Stage Attempt ID":0,"Stage Name":"collect at SparkSQLExample.scala:230","Number of Tasks":2,"RDD Info":[{"RDD ID":63,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"162\",\"name\":\"mapPartitionsInternal\"}","Callsite":"collect at SparkSQLExample.scala:230","Parent IDs":[62],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"118\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:200","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":62,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"154\",\"name\":\"WholeStageCodegen\"}","Callsite":"collect at SparkSQLExample.scala:230","Parent IDs":[61],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":61,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"161\",\"name\":\"Scan\"}","Callsite":"collect at SparkSQLExample.scala:230","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"117\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:199","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.collect(Dataset.scala:2708)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:230)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788475755,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.rdd.scope":"{\"id\":\"163\",\"name\":\"collect\"}","spark.rdd.scope.noOverride":"true","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"16","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerTaskStart","Stage ID":25,"Stage Attempt ID":0,"Task Info":{"Task ID":217,"Index":0,"Attempt":0,"Launch Time":1547788475761,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":25,"Stage Attempt ID":0,"Task Info":{"Task ID":218,"Index":1,"Attempt":0,"Launch Time":1547788475867,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":25,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":217,"Index":0,"Attempt":0,"Launch Time":1547788475761,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788475868,"Failed":false,"Killed":false,"Accumulables":[{"ID":733,"Name":"duration total (min, med, max)","Update":"6","Value":"5","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":735,"Name":"number of output rows","Update":"2","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":758,"Name":"internal.metrics.input.recordsRead","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":757,"Name":"internal.metrics.input.bytesRead","Update":32,"Value":32,"Internal":true,"Count Failed Values":true},{"ID":740,"Name":"internal.metrics.resultSize","Update":1390,"Value":1390,"Internal":true,"Count Failed Values":true},{"ID":739,"Name":"internal.metrics.executorCpuTime","Update":78410986,"Value":78410986,"Internal":true,"Count Failed Values":true},{"ID":738,"Name":"internal.metrics.executorRunTime","Update":86,"Value":86,"Internal":true,"Count Failed Values":true},{"ID":737,"Name":"internal.metrics.executorDeserializeCpuTime","Update":9743696,"Value":9743696,"Internal":true,"Count Failed Values":true},{"ID":736,"Name":"internal.metrics.executorDeserializeTime","Update":14,"Value":14,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":14,"Executor Deserialize CPU Time":9743696,"Executor Run Time":86,"Executor CPU Time":78410986,"Result Size":1390,"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":32,"Records Read":2},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":25,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":218,"Index":1,"Attempt":0,"Launch Time":1547788475867,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788476471,"Failed":false,"Killed":false,"Accumulables":[{"ID":733,"Name":"duration total (min, med, max)","Update":"582","Value":"587","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":734,"Name":"number of output rows","Update":"1","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":735,"Name":"number of output rows","Update":"1","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":758,"Name":"internal.metrics.input.recordsRead","Update":1,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":757,"Name":"internal.metrics.input.bytesRead","Update":16,"Value":48,"Internal":true,"Count Failed Values":true},{"ID":740,"Name":"internal.metrics.resultSize","Update":1439,"Value":2829,"Internal":true,"Count Failed Values":true},{"ID":739,"Name":"internal.metrics.executorCpuTime","Update":575036156,"Value":653447142,"Internal":true,"Count Failed Values":true},{"ID":738,"Name":"internal.metrics.executorRunTime","Update":591,"Value":677,"Internal":true,"Count Failed Values":true},{"ID":737,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3688735,"Value":13432431,"Internal":true,"Count Failed Values":true},{"ID":736,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":19,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":3688735,"Executor Run Time":591,"Executor CPU Time":575036156,"Result Size":1439,"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":16,"Records Read":1},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":25,"Stage Attempt ID":0,"Stage Name":"collect at SparkSQLExample.scala:230","Number of Tasks":2,"RDD Info":[{"RDD ID":63,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"162\",\"name\":\"mapPartitionsInternal\"}","Callsite":"collect at SparkSQLExample.scala:230","Parent IDs":[62],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":52,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"118\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:200","Parent IDs":[51],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":62,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"154\",\"name\":\"WholeStageCodegen\"}","Callsite":"collect at SparkSQLExample.scala:230","Parent IDs":[61],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":49,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":61,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"161\",\"name\":\"Scan\"}","Callsite":"collect at SparkSQLExample.scala:230","Parent IDs":[52],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":50,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"116\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:197","Parent IDs":[49],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":51,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"117\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:199","Parent IDs":[50],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.collect(Dataset.scala:2708)\norg.apache.spark.examples.sql.SparkSQLExample$.runInferSchemaExample(SparkSQLExample.scala:230)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:51)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788475755,"Completion Time":1547788476473,"Accumulables":[{"ID":738,"Name":"internal.metrics.executorRunTime","Value":677,"Internal":true,"Count Failed Values":true},{"ID":735,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":737,"Name":"internal.metrics.executorDeserializeCpuTime","Value":13432431,"Internal":true,"Count Failed Values":true},{"ID":758,"Name":"internal.metrics.input.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":740,"Name":"internal.metrics.resultSize","Value":2829,"Internal":true,"Count Failed Values":true},{"ID":734,"Name":"number of output rows","Value":"1","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":757,"Name":"internal.metrics.input.bytesRead","Value":48,"Internal":true,"Count Failed Values":true},{"ID":733,"Name":"duration total (min, med, max)","Value":"587","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":736,"Name":"internal.metrics.executorDeserializeTime","Value":19,"Internal":true,"Count Failed Values":true},{"ID":739,"Name":"internal.metrics.executorCpuTime","Value":653447142,"Internal":true,"Count Failed Values":true}]}} -{"Event":"SparkListenerJobEnd","Job ID":20,"Completion Time":1547788476474,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":16,"time":1547788476729} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":17,"description":"createOrReplaceTempView at SparkSQLExample.scala:258","details":"org.apache.spark.sql.Dataset.createOrReplaceTempView(Dataset.scala:3021)\norg.apache.spark.examples.sql.SparkSQLExample$.runProgrammaticSchemaExample(SparkSQLExample.scala:258)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:52)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\nCreateViewCommand `people`, false, true, LocalTempView\n +- LogicalRDD [name#206, age#207], false\n\n== Analyzed Logical Plan ==\n\nCreateViewCommand `people`, false, true, LocalTempView\n +- LogicalRDD [name#206, age#207], false\n\n== Optimized Logical Plan ==\nCreateViewCommand `people`, false, true, LocalTempView\n +- LogicalRDD [name#206, age#207], false\n\n== Physical Plan ==\nExecute CreateViewCommand\n +- CreateViewCommand `people`, false, true, LocalTempView\n +- LogicalRDD [name#206, age#207], false\n","sparkPlanInfo":{"nodeName":"Execute CreateViewCommand","simpleString":"Execute CreateViewCommand","children":[],"metadata":{},"metrics":[]},"time":1547788476801} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":17,"time":1547788476802} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":18,"description":"show at SparkSQLExample.scala:265","details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runProgrammaticSchemaExample(SparkSQLExample.scala:265)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:52)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","physicalPlanDescription":"== Parsed Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(value#214 as string) AS value#218]\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#214]\n +- MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2583/185651170@6b9d8f7, interface org.apache.spark.sql.Row, [StructField(name,StringType,true)], obj#213: java.lang.String\n +- DeserializeToObject createexternalrow(name#206.toString, StructField(name,StringType,true)), obj#212: org.apache.spark.sql.Row\n +- Project [name#206]\n +- SubqueryAlias `people`\n +- LogicalRDD [name#206, age#207], false\n\n== Analyzed Logical Plan ==\nvalue: string\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(value#214 as string) AS value#218]\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#214]\n +- MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2583/185651170@6b9d8f7, interface org.apache.spark.sql.Row, [StructField(name,StringType,true)], obj#213: java.lang.String\n +- DeserializeToObject createexternalrow(name#206.toString, StructField(name,StringType,true)), obj#212: org.apache.spark.sql.Row\n +- Project [name#206]\n +- SubqueryAlias `people`\n +- LogicalRDD [name#206, age#207], false\n\n== Optimized Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#214]\n +- MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2583/185651170@6b9d8f7, interface org.apache.spark.sql.Row, [StructField(name,StringType,true)], obj#213: java.lang.String\n +- DeserializeToObject createexternalrow(name#206.toString, StructField(name,StringType,true)), obj#212: org.apache.spark.sql.Row\n +- Project [name#206]\n +- LogicalRDD [name#206, age#207], false\n\n== Physical Plan ==\nCollectLimit 21\n+- *(1) SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#214]\n +- *(1) MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2583/185651170@6b9d8f7, obj#213: java.lang.String\n +- *(1) DeserializeToObject createexternalrow(name#206.toString, StructField(name,StringType,true)), obj#212: org.apache.spark.sql.Row\n +- *(1) Project [name#206]\n +- *(1) Scan ExistingRDD[name#206,age#207]\n","sparkPlanInfo":{"nodeName":"CollectLimit","simpleString":"CollectLimit 21","children":[{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"SerializeFromObject","simpleString":"SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#214]","children":[{"nodeName":"MapElements","simpleString":"MapElements org.apache.spark.examples.sql.SparkSQLExample$$$Lambda$2583/185651170@6b9d8f7, obj#213: java.lang.String","children":[{"nodeName":"DeserializeToObject","simpleString":"DeserializeToObject createexternalrow(name#206.toString, StructField(name,StringType,true)), obj#212: org.apache.spark.sql.Row","children":[{"nodeName":"Project","simpleString":"Project [name#206]","children":[{"nodeName":"Scan ExistingRDD","simpleString":"Scan ExistingRDD[name#206,age#207]","children":[],"metadata":{},"metrics":[{"name":"number of output rows","accumulatorId":772,"metricType":"sum"}]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[]}],"metadata":{},"metrics":[{"name":"duration total (min, med, max)","accumulatorId":771,"metricType":"timing"}]}],"metadata":{},"metrics":[{"name":"shuffle records written","accumulatorId":769,"metricType":"sum"},{"name":"shuffle write time total (min, med, max)","accumulatorId":770,"metricType":"nsTiming"},{"name":"records read","accumulatorId":767,"metricType":"sum"},{"name":"local bytes read total (min, med, max)","accumulatorId":765,"metricType":"size"},{"name":"fetch wait time total (min, med, max)","accumulatorId":766,"metricType":"timing"},{"name":"remote bytes read total (min, med, max)","accumulatorId":763,"metricType":"size"},{"name":"local blocks read","accumulatorId":762,"metricType":"sum"},{"name":"remote blocks read","accumulatorId":761,"metricType":"sum"},{"name":"remote bytes read to disk total (min, med, max)","accumulatorId":764,"metricType":"size"},{"name":"shuffle bytes written total (min, med, max)","accumulatorId":768,"metricType":"size"}]},"time":1547788476854} -{"Event":"SparkListenerJobStart","Job ID":21,"Submission Time":1547788476899,"Stage Infos":[{"Stage ID":26,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:265","Number of Tasks":1,"RDD Info":[{"RDD ID":71,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"179\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[70],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":69,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"172\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[68],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":70,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"178\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[69],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":65,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"168\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:239","Parent IDs":[64],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":67,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"170\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:252","Parent IDs":[66],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":64,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"168\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:239","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":66,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"169\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:251","Parent IDs":[65],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":68,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"171\",\"name\":\"map\"}","Callsite":"createDataFrame at SparkSQLExample.scala:255","Parent IDs":[67],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runProgrammaticSchemaExample(SparkSQLExample.scala:265)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:52)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[26],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"18","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":26,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:265","Number of Tasks":1,"RDD Info":[{"RDD ID":71,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"179\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[70],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":69,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"172\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[68],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":70,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"178\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[69],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":65,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"168\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:239","Parent IDs":[64],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":67,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"170\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:252","Parent IDs":[66],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":64,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"168\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:239","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":66,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"169\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:251","Parent IDs":[65],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":68,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"171\",\"name\":\"map\"}","Callsite":"createDataFrame at SparkSQLExample.scala:255","Parent IDs":[67],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runProgrammaticSchemaExample(SparkSQLExample.scala:265)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:52)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788476901,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"18","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerTaskStart","Stage ID":26,"Stage Attempt ID":0,"Task Info":{"Task ID":219,"Index":0,"Attempt":0,"Launch Time":1547788476909,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":26,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":219,"Index":0,"Attempt":0,"Launch Time":1547788476909,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788479653,"Failed":false,"Killed":false,"Accumulables":[{"ID":771,"Name":"duration total (min, med, max)","Update":"2512","Value":"2511","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":772,"Name":"number of output rows","Update":"2","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":795,"Name":"internal.metrics.input.recordsRead","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":794,"Name":"internal.metrics.input.bytesRead","Update":32,"Value":32,"Internal":true,"Count Failed Values":true},{"ID":778,"Name":"internal.metrics.jvmGCTime","Update":37,"Value":37,"Internal":true,"Count Failed Values":true},{"ID":777,"Name":"internal.metrics.resultSize","Update":1240,"Value":1240,"Internal":true,"Count Failed Values":true},{"ID":776,"Name":"internal.metrics.executorCpuTime","Update":2467792536,"Value":2467792536,"Internal":true,"Count Failed Values":true},{"ID":775,"Name":"internal.metrics.executorRunTime","Update":2594,"Value":2594,"Internal":true,"Count Failed Values":true},{"ID":774,"Name":"internal.metrics.executorDeserializeCpuTime","Update":136342693,"Value":136342693,"Internal":true,"Count Failed Values":true},{"ID":773,"Name":"internal.metrics.executorDeserializeTime","Update":144,"Value":144,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":144,"Executor Deserialize CPU Time":136342693,"Executor Run Time":2594,"Executor CPU Time":2467792536,"Result Size":1240,"JVM GC Time":37,"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":32,"Records Read":2},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":26,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:265","Number of Tasks":1,"RDD Info":[{"RDD ID":71,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"179\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[70],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":69,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"172\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[68],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":70,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"178\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[69],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":65,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"168\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:239","Parent IDs":[64],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":67,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"170\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:252","Parent IDs":[66],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":64,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"168\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:239","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":66,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"169\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:251","Parent IDs":[65],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":68,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"171\",\"name\":\"map\"}","Callsite":"createDataFrame at SparkSQLExample.scala:255","Parent IDs":[67],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runProgrammaticSchemaExample(SparkSQLExample.scala:265)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:52)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788476901,"Completion Time":1547788479654,"Accumulables":[{"ID":774,"Name":"internal.metrics.executorDeserializeCpuTime","Value":136342693,"Internal":true,"Count Failed Values":true},{"ID":795,"Name":"internal.metrics.input.recordsRead","Value":2,"Internal":true,"Count Failed Values":true},{"ID":777,"Name":"internal.metrics.resultSize","Value":1240,"Internal":true,"Count Failed Values":true},{"ID":771,"Name":"duration total (min, med, max)","Value":"2511","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":773,"Name":"internal.metrics.executorDeserializeTime","Value":144,"Internal":true,"Count Failed Values":true},{"ID":794,"Name":"internal.metrics.input.bytesRead","Value":32,"Internal":true,"Count Failed Values":true},{"ID":776,"Name":"internal.metrics.executorCpuTime","Value":2467792536,"Internal":true,"Count Failed Values":true},{"ID":775,"Name":"internal.metrics.executorRunTime","Value":2594,"Internal":true,"Count Failed Values":true},{"ID":778,"Name":"internal.metrics.jvmGCTime","Value":37,"Internal":true,"Count Failed Values":true},{"ID":772,"Name":"number of output rows","Value":"2","Internal":true,"Count Failed Values":true,"Metadata":"sql"}]}} -{"Event":"SparkListenerJobEnd","Job ID":21,"Completion Time":1547788479655,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerJobStart","Job ID":22,"Submission Time":1547788479665,"Stage Infos":[{"Stage ID":27,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:265","Number of Tasks":1,"RDD Info":[{"RDD ID":71,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"179\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[70],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":69,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"172\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[68],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":70,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"178\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[69],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":65,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"168\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:239","Parent IDs":[64],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":67,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"170\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:252","Parent IDs":[66],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":64,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"168\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:239","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":66,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"169\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:251","Parent IDs":[65],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":68,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"171\",\"name\":\"map\"}","Callsite":"createDataFrame at SparkSQLExample.scala:255","Parent IDs":[67],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runProgrammaticSchemaExample(SparkSQLExample.scala:265)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:52)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[27],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"18","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":27,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:265","Number of Tasks":1,"RDD Info":[{"RDD ID":71,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"179\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[70],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":69,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"172\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[68],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":70,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"178\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[69],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":65,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"168\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:239","Parent IDs":[64],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":67,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"170\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:252","Parent IDs":[66],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":64,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"168\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:239","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":66,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"169\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:251","Parent IDs":[65],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":68,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"171\",\"name\":\"map\"}","Callsite":"createDataFrame at SparkSQLExample.scala:255","Parent IDs":[67],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runProgrammaticSchemaExample(SparkSQLExample.scala:265)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:52)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788479666,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.sql.execution.id":"18","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerTaskStart","Stage ID":27,"Stage Attempt ID":0,"Task Info":{"Task ID":220,"Index":0,"Attempt":0,"Launch Time":1547788479676,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":27,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":220,"Index":0,"Attempt":0,"Launch Time":1547788479676,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788479710,"Failed":false,"Killed":false,"Accumulables":[{"ID":771,"Name":"duration total (min, med, max)","Update":"6","Value":"2517","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":772,"Name":"number of output rows","Update":"1","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":820,"Name":"internal.metrics.input.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":819,"Name":"internal.metrics.input.bytesRead","Update":16,"Value":16,"Internal":true,"Count Failed Values":true},{"ID":802,"Name":"internal.metrics.resultSize","Update":1180,"Value":1180,"Internal":true,"Count Failed Values":true},{"ID":801,"Name":"internal.metrics.executorCpuTime","Update":12086508,"Value":12086508,"Internal":true,"Count Failed Values":true},{"ID":800,"Name":"internal.metrics.executorRunTime","Update":14,"Value":14,"Internal":true,"Count Failed Values":true},{"ID":799,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8317054,"Value":8317054,"Internal":true,"Count Failed Values":true},{"ID":798,"Name":"internal.metrics.executorDeserializeTime","Update":14,"Value":14,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":14,"Executor Deserialize CPU Time":8317054,"Executor Run Time":14,"Executor CPU Time":12086508,"Result Size":1180,"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":16,"Records Read":1},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":27,"Stage Attempt ID":0,"Stage Name":"show at SparkSQLExample.scala:265","Number of Tasks":1,"RDD Info":[{"RDD ID":71,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"179\",\"name\":\"map\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[70],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":69,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"172\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[68],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":70,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"178\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at SparkSQLExample.scala:265","Parent IDs":[69],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":65,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"168\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:239","Parent IDs":[64],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":67,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"170\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:252","Parent IDs":[66],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":64,"Name":"examples/src/main/resources/people.txt","Scope":"{\"id\":\"168\",\"name\":\"textFile\"}","Callsite":"textFile at SparkSQLExample.scala:239","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":66,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"169\",\"name\":\"map\"}","Callsite":"map at SparkSQLExample.scala:251","Parent IDs":[65],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":68,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"171\",\"name\":\"map\"}","Callsite":"createDataFrame at SparkSQLExample.scala:255","Parent IDs":[67],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:721)\norg.apache.spark.examples.sql.SparkSQLExample$.runProgrammaticSchemaExample(SparkSQLExample.scala:265)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:52)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788479666,"Completion Time":1547788479711,"Accumulables":[{"ID":819,"Name":"internal.metrics.input.bytesRead","Value":16,"Internal":true,"Count Failed Values":true},{"ID":801,"Name":"internal.metrics.executorCpuTime","Value":12086508,"Internal":true,"Count Failed Values":true},{"ID":771,"Name":"duration total (min, med, max)","Value":"2517","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":800,"Name":"internal.metrics.executorRunTime","Value":14,"Internal":true,"Count Failed Values":true},{"ID":820,"Name":"internal.metrics.input.recordsRead","Value":1,"Internal":true,"Count Failed Values":true},{"ID":802,"Name":"internal.metrics.resultSize","Value":1180,"Internal":true,"Count Failed Values":true},{"ID":772,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":799,"Name":"internal.metrics.executorDeserializeCpuTime","Value":8317054,"Internal":true,"Count Failed Values":true},{"ID":798,"Name":"internal.metrics.executorDeserializeTime","Value":14,"Internal":true,"Count Failed Values":true}]}} -{"Event":"SparkListenerJobEnd","Job ID":22,"Completion Time":1547788479711,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":18,"time":1547788479712} {"Event":"SparkListenerApplicationEnd","Timestamp":1547788479714} From 782fe7f58300f1e83b439d843a8befff2e230a32 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Fri, 18 Jan 2019 22:27:42 +0900 Subject: [PATCH 12/28] Remove @JsonIgnore which doesn't work actually --- core/src/main/scala/org/apache/spark/status/api/v1/api.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala index 2a3f501cc0283..b6b544783ebbe 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala @@ -21,7 +21,7 @@ import java.util.Date import scala.xml.{NodeSeq, Text} -import com.fasterxml.jackson.annotation.{JsonIgnore, JsonIgnoreProperties} +import com.fasterxml.jackson.annotation.JsonIgnoreProperties import com.fasterxml.jackson.core.{JsonGenerator, JsonParser} import com.fasterxml.jackson.core.`type`.TypeReference import com.fasterxml.jackson.databind.{DeserializationContext, JsonDeserializer, JsonSerializer, SerializerProvider} @@ -107,7 +107,7 @@ class ExecutorSummary private[spark]( @JsonSerialize(using = classOf[ExecutorMetricsJsonSerializer]) @JsonDeserialize(using = classOf[ExecutorMetricsJsonDeserializer]) val peakMemoryMetrics: Option[ExecutorMetrics], - @JsonIgnore val attributes: Map[String, String]) { + val attributes: Map[String, String]) { def replaceExecutorLogs(newExecutorLogs: Map[String, String]): ExecutorSummary = { new ExecutorSummary(id, hostPort, isActive, rddBlocks, memoryUsed, diskUsed, totalCores, From b5d4e5faf5072bc4dc93d59b71833cc20eaf4392 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Sat, 19 Jan 2019 08:12:56 +0900 Subject: [PATCH 13/28] Add incomplete app to test target for custom log url as well as others --- .../application_list_json_expectation.json | 16 +++ ...on_apply_custom_log_urls_expectation.json} | 0 ...son_apply_custom_log_urls_expectation.json | 126 ++++++++++++++++++ .../minDate_app_list_json_expectation.json | 16 +++ .../minEndDate_app_list_json_expectation.json | 16 +++ .../running_app_list_json_expectation.json | 18 ++- .../application_1547723113049_0006 | 12 ++ .../deploy/history/HistoryServerSuite.scala | 48 ++++--- dev/.rat-excludes | 1 + 9 files changed, 232 insertions(+), 21 deletions(-) rename core/src/test/resources/HistoryServerExpectations/{executor_list_json_apply_custom_log_urls_expectation.json => completed_app_executor_list_json_apply_custom_log_urls_expectation.json} (100%) create mode 100644 core/src/test/resources/HistoryServerExpectations/incomplete_app_executor_list_json_apply_custom_log_urls_expectation.json create mode 100644 core/src/test/resources/spark-events/application_1547723113049_0006 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 b7b67eea7b298..20a8724e3c06d 100644 --- a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json @@ -220,4 +220,20 @@ "startTimeEpoch" : 1422981758277, "endTimeEpoch" : 1422981766912 } ] +}, { + "id" : "application_1547723113049_0006", + "name" : "Spark SQL basic example", + "attempts" : [ { + "attemptId" : "1", + "startTime" : "2019-01-18T05:14:14.200GMT", + "endTime" : "1969-12-31T23:59:59.999GMT", + "lastUpdated" : "", + "duration" : 0, + "sparkUser" : "spark", + "completed" : false, + "appSparkVersion" : "3.0.0-SNAPSHOT", + "startTimeEpoch" : 1547788454200, + "endTimeEpoch" : -1, + "lastUpdatedEpoch" : 0 + } ] } ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_json_apply_custom_log_urls_expectation.json b/core/src/test/resources/HistoryServerExpectations/completed_app_executor_list_json_apply_custom_log_urls_expectation.json similarity index 100% rename from core/src/test/resources/HistoryServerExpectations/executor_list_json_apply_custom_log_urls_expectation.json rename to core/src/test/resources/HistoryServerExpectations/completed_app_executor_list_json_apply_custom_log_urls_expectation.json diff --git a/core/src/test/resources/HistoryServerExpectations/incomplete_app_executor_list_json_apply_custom_log_urls_expectation.json b/core/src/test/resources/HistoryServerExpectations/incomplete_app_executor_list_json_apply_custom_log_urls_expectation.json new file mode 100644 index 0000000000000..4494767a45a63 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/incomplete_app_executor_list_json_apply_custom_log_urls_expectation.json @@ -0,0 +1,126 @@ +[ + { + "id":"driver", + "hostPort":"node2:40675", + "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":428762726, + "addTime":"2019-01-18T05:14:05.481GMT", + "executorLogs":{ + "stdout":"http://node2:8042/node/containerlogs/container_e01_1547723113049_0005_01_000001/spark/stdout?start=-4096", + "stderr":"http://node2:8042/node/containerlogs/container_e01_1547723113049_0005_01_000001/spark/stderr?start=-4096" + }, + "memoryMetrics":{ + "usedOnHeapStorageMemory":0, + "usedOffHeapStorageMemory":0, + "totalOnHeapStorageMemory":428762726, + "totalOffHeapStorageMemory":0 + }, + "blacklistedInStages":[ + + ], + "attributes":{ + + } + }, + { + "id":"2", + "hostPort":"node4:44875", + "isActive":true, + "rddBlocks":0, + "memoryUsed":0, + "diskUsed":0, + "totalCores":1, + "maxTasks":1, + "activeTasks":0, + "failedTasks":0, + "completedTasks":0, + "totalTasks":0, + "totalDuration":0, + "totalGCTime":0, + "totalInputBytes":0, + "totalShuffleRead":0, + "totalShuffleWrite":0, + "isBlacklisted":false, + "maxMemory":384093388, + "addTime":"2019-01-18T05:14:15.343GMT", + "executorLogs":{ + "stderr":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000003/stderr)", + "stdout":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000003/stdout)" + }, + "memoryMetrics":{ + "usedOnHeapStorageMemory":0, + "usedOffHeapStorageMemory":0, + "totalOnHeapStorageMemory":384093388, + "totalOffHeapStorageMemory":0 + }, + "blacklistedInStages":[ + + ], + "attributes":{ + "NODE_HTTP_ADDRESS":"node4:8042", + "USER":"spark", + "LOG_FILES":"stderr,stdout", + "CLUSTER_ID":"yarn-cluster", + "HTTP_SCHEME":"http://", + "CONTAINER_ID":"container_e01_1547723113049_0005_01_000003" + } + }, + { + "id":"1", + "hostPort":"node3:35087", + "isActive":true, + "rddBlocks":0, + "memoryUsed":0, + "diskUsed":0, + "totalCores":1, + "maxTasks":1, + "activeTasks":0, + "failedTasks":0, + "completedTasks":1, + "totalTasks":1, + "totalDuration":4025, + "totalGCTime":159, + "totalInputBytes":73, + "totalShuffleRead":0, + "totalShuffleWrite":0, + "isBlacklisted":false, + "maxMemory":384093388, + "addTime":"2019-01-18T05:14:14.209GMT", + "executorLogs":{ + "stderr":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000002/stderr)", + "stdout":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000002/stdout)" + }, + "memoryMetrics":{ + "usedOnHeapStorageMemory":0, + "usedOffHeapStorageMemory":0, + "totalOnHeapStorageMemory":384093388, + "totalOffHeapStorageMemory":0 + }, + "blacklistedInStages":[ + + ], + "attributes":{ + "NODE_HTTP_ADDRESS":"node3:8042", + "USER":"spark", + "LOG_FILES":"stderr,stdout", + "CLUSTER_ID":"yarn-cluster", + "HTTP_SCHEME":"http://", + "CONTAINER_ID":"container_e01_1547723113049_0005_01_000002" + } + } +] \ No newline at end of file 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 681a8bc781260..2c679a5bb3e05 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 @@ -190,4 +190,20 @@ "startTimeEpoch" : 1425081758277, "endTimeEpoch" : 1425081766912 } ] +}, { + "id" : "application_1547723113049_0006", + "name" : "Spark SQL basic example", + "attempts" : [ { + "attemptId" : "1", + "startTime" : "2019-01-18T05:14:14.200GMT", + "endTime" : "1969-12-31T23:59:59.999GMT", + "lastUpdated" : "", + "duration" : 0, + "sparkUser" : "spark", + "completed" : false, + "appSparkVersion" : "3.0.0-SNAPSHOT", + "startTimeEpoch" : 1547788454200, + "endTimeEpoch" : -1, + "lastUpdatedEpoch" : 0 + } ] } ] 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 a50b74ee50e55..17b3e728e4848 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 @@ -147,4 +147,20 @@ "startTimeEpoch" : 1430917380880, "endTimeEpoch" : 1430917380890 } ] +}, { + "id" : "application_1547723113049_0006", + "name" : "Spark SQL basic example", + "attempts" : [ { + "attemptId" : "1", + "startTime" : "2019-01-18T05:14:14.200GMT", + "endTime" : "1969-12-31T23:59:59.999GMT", + "lastUpdated" : "", + "duration" : 0, + "sparkUser" : "spark", + "completed" : false, + "appSparkVersion" : "3.0.0-SNAPSHOT", + "startTimeEpoch" : 1547788454200, + "endTimeEpoch" : -1, + "lastUpdatedEpoch" : 0 + } ] } ] diff --git a/core/src/test/resources/HistoryServerExpectations/running_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/running_app_list_json_expectation.json index 1e3ec7217afba..b766580def925 100644 --- a/core/src/test/resources/HistoryServerExpectations/running_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/running_app_list_json_expectation.json @@ -1 +1,17 @@ -[ ] +[ { + "id" : "application_1547723113049_0006", + "name" : "Spark SQL basic example", + "attempts" : [ { + "attemptId" : "1", + "startTime" : "2019-01-18T05:14:14.200GMT", + "endTime" : "1969-12-31T23:59:59.999GMT", + "lastUpdated" : "", + "duration" : 0, + "sparkUser" : "spark", + "completed" : false, + "appSparkVersion" : "3.0.0-SNAPSHOT", + "startTimeEpoch" : 1547788454200, + "endTimeEpoch" : -1, + "lastUpdatedEpoch" : 0 + } ] +} ] diff --git a/core/src/test/resources/spark-events/application_1547723113049_0006 b/core/src/test/resources/spark-events/application_1547723113049_0006 new file mode 100644 index 0000000000000..10b9c6eb10f7f --- /dev/null +++ b/core/src/test/resources/spark-events/application_1547723113049_0006 @@ -0,0 +1,12 @@ +{"Event":"SparkListenerLogStart","Spark Version":"3.0.0-SNAPSHOT"} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"node2","Port":40675},"Maximum Memory":428762726,"Timestamp":1547788445481,"Maximum Onheap Memory":428762726,"Maximum Offheap Memory":0} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre","Java Version":"1.8.0_161 (Oracle Corporation)","Scala Version":"version 2.12.8"},"Spark Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0006","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.scheduler.mode":"FIFO","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0006","spark.app.id":"application_1547723113049_0006","spark.hadoop.yarn.timeline-service.enabled":"false"},"System Properties":{"java.io.tmpdir":"/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_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/yarn","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/amd64","user.dir":"/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001","java.library.path":"/usr/java/packages/lib/amd64:/usr/lib64:/lib64:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.161-b14","jetty.git.hash":"27208684755d94a92186989f695db2d7b21ebc51","java.endorsed.dirs":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/endorsed","java.runtime.version":"1.8.0_161-b14","java.vm.info":"mixed mode","java.ext.dirs":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"OpenJDK Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/resources.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/rt.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/sunrsasign.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/jsse.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/jce.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/charsets.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/jfr.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/classes","file.encoding":"UTF-8","user.timezone":"UTC","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"4.18.5-1.el7.elrepo.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":"yarn","hdp.version":"2.6.5.1002-73","java.vm.name":"OpenJDK 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.yarn.ApplicationMaster --class org.apache.spark.examples.sql.SparkSQLExample --jar file:/home/spark/spark-3.0.0-snapshot-spark-26311-v2/examples/jars/spark-examples_2.12-3.0.0-SNAPSHOT.jar --properties-file /hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_conf__/__spark_conf__.properties --dist-cache-conf /hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_conf__/__spark_dist_cache__.properties","java.home":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre","java.version":"1.8.0_161","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/hadoop-yarn-client/lib/commons-digester-1.8.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-applications-distributedshell-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-compiler-3.0.11.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-hadoop-bundle-1.6.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/lz4-java-1.5.0.jar":"System Classpath","/hadoop/lib/commons-digester-1.8.jar":"System Classpath","/hadoop-yarn-client/lib/netty-3.6.2.Final.jar":"System Classpath","/hadoop/lib/curator-client-2.7.1.jar":"System Classpath","/hadoop/lib/log4j-1.2.17.jar":"System Classpath","/hadoop/lib/paranamer-2.3.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-codec-1.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hk2-api-2.4.0-b34.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/orc-core-1.5.4-nohive.jar":"System Classpath","/hadoop-yarn-client/lib/curator-client-2.7.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop/hadoop-common.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-cli-1.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-hdfs-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/aopalliance-1.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-guava-2.22.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/stax-api-1.0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/snappy-java-1.1.7.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/univocity-parsers-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/azure-storage-5.4.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/netty-all-4.1.30.Final.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-container-servlet-core-2.22.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/macro-compat_2.12-1.1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-httpclient-3.1.jar":"System Classpath","/hadoop/lib/httpcore-4.4.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/hadoop/lib/java-xmlbuilder-0.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jodd-core-3.5.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/breeze-macros_2.12-0.13.2.jar":"System Classpath","/hadoop-yarn-client/lib/zookeeper-3.4.6.jar":"System Classpath","/hadoop/lib/curator-framework-2.7.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jul-to-slf4j-1.7.16.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/xz-1.5.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/antlr-2.7.7.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/json4s-scalap_2.12-3.5.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/apache-log4j-extras-1.2.17.jar":"System Classpath","/hadoop-hdfs-client/lib/htrace-core-3.1.0-incubating.jar":"System Classpath","/hadoop/hadoop-azure.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-nodemanager.jar":"System Classpath","/hadoop-yarn-client/lib/servlet-api-2.5.jar":"System Classpath","/hadoop-yarn-client/lib/guava-11.0.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/arpack_combined_all-0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/arrow-format-0.10.0.jar":"System Classpath","/hadoop-yarn-client/lib/guice-3.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/scala-library-2.12.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-container-servlet-2.22.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/xercesImpl-2.9.1.jar":"System Classpath","/hadoop/hadoop-azure-datalake-2.7.3.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-common.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/yarn/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-auth-2.7.4.jar":"System Classpath","/hadoop/lib/activation-1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jline-2.14.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/yarn/lib/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hive-metastore-1.2.1.spark2.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-applicationhistoryservice.jar":"System Classpath","/hadoop/lib/ojdbc6.jar":"System Classpath","/hadoop/lib/commons-collections-3.2.2.jar":"System Classpath","/hadoop-hdfs-client/lib/servlet-api-2.5.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/joda-time-2.9.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-annotations-2.9.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-mllib-local_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/leveldbjni-all-1.8.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-common-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/json4s-jackson_2.12-3.5.3.jar":"System Classpath","/hadoop/conf":"System Classpath","/hadoop/hadoop-common-2.7.3-tests.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-timeline-pluginstorage-2.7.3.jar":"System Classpath","/hadoop/lib/junit-4.11.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs-2.7.3-tests.jar":"System Classpath","/hadoop-yarn-client/lib/httpcore-4.4.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-unsafe_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/metrics-jvm-3.1.5.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/objenesis-2.5.1.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-nodemanager-2.7.3.jar":"System Classpath","/hadoop/hadoop-common-tests.jar":"System Classpath","/hadoop-hdfs-client/lib/jackson-databind-2.2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-graphx_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-hdfs-client/lib/asm-3.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/kryo-shaded-4.0.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/compress-lzf-1.0.3.jar":"System Classpath","/hadoop/lib/azure-storage-5.4.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/avro-mapred-1.8.2-hadoop2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/scala-xml_2.12-1.0.5.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-applications-distributedshell.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jaxb-api-2.2.2.jar":"System Classpath","/hadoop/lib/commons-net-3.1.jar":"System Classpath","/hadoop-yarn-client/lib/java-xmlbuilder-0.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/paranamer-2.8.jar":"System Classpath","/hadoop/lib/commons-logging-1.1.3.jar":"System Classpath","/hadoop-yarn-client/lib/log4j-1.2.17.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/mapreduce/*":"System Classpath","/hadoop/lib/commons-codec-1.4.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs-nfs-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/commons-lang3-3.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-math3-3.4.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-network-common_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jetty-6.1.26.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/javolution-5.5.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jcl-over-slf4j-1.7.16.jar":"System Classpath","/hadoop/lib/slf4j-api-1.7.10.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-common-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-core-asl-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hk2-utils-2.4.0-b34.jar":"System Classpath","/hadoop/lib/mockito-all-1.8.5.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-client.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/common/lib/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hppc-0.7.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/shapeless_2.12-2.3.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-mapreduce-client-common-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/log4j-1.2.17.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-module-scala_2.12-2.9.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-module-paranamer-2.9.6.jar":"System Classpath","/hadoop-hdfs-client/lib/jetty-6.1.26.hwx.jar":"System Classpath","/hadoop/hadoop-aws.jar":"System Classpath","/hadoop/lib/commons-cli-1.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-codec-1.10.jar":"System Classpath","/hadoop/lib/commons-compress-1.4.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-server-2.22.2.jar":"System Classpath","/hadoop/lib/avro-1.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-xc-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-dbcp-1.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-beanutils-1.7.0.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-jaxrs-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/api-asn1-api-1.0.0-M20.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-client-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/javax.inject-2.4.0-b34.jar":"System Classpath","/hadoop/gcs-connector-1.8.1-shaded.jar":"System Classpath","/hadoop-hdfs-client/lib/jersey-core-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/metrics-json-3.1.5.jar":"System Classpath","/hadoop/lib/jackson-core-2.2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jpam-1.1.jar":"System Classpath","/hadoop/lib/jersey-core-1.9.jar":"System Classpath","/hadoop/lib/snappy-java-1.0.4.1.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-lang-2.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/slf4j-log4j12-1.7.16.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/osgi-resource-locator-1.0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/api-asn1-api-1.0.0-M20.jar":"System Classpath","/hadoop-yarn-client/lib/nimbus-jose-jwt-4.41.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-databind-2.9.6.jar":"System Classpath","/hadoop-hdfs-client/lib/xml-apis-1.3.04.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-client-2.22.2.jar":"System Classpath","/hadoop/hadoop-nfs-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hive-jdbc-1.2.1.spark2.jar":"System Classpath","/hadoop/lib/jackson-databind-2.2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/eigenbase-properties-1.1.5.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/machinist_2.12-0.6.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/tools/lib/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-jackson-1.10.0.jar":"System Classpath","/hadoop-yarn-client/lib/jetty-util-6.1.26.hwx.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-digester-1.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/calcite-avatica-1.2.0-incubating.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/oro-2.0.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hive-cli-1.2.1.spark2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-hive-thriftserver_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-applications-unmanaged-am-launcher-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/JavaEWAH-0.3.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/super-csv-2.2.0.jar":"System Classpath","/hadoop/lib/ranger-yarn-plugin-shim-0.7.0.jar":"System Classpath","/hadoop-yarn-client/lib/jetty-6.1.26.hwx.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/metrics-core-3.1.5.jar":"System Classpath","/hadoop-yarn-client/lib/commons-compress-1.4.1.jar":"System Classpath","/hadoop-hdfs-client/lib/netty-all-4.0.52.Final.jar":"System Classpath","/hadoop/hadoop-auth.jar":"System Classpath","/hadoop-yarn-client/lib/javax.inject-1.jar":"System Classpath","/hadoop/lib/jettison-1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jetty-util-6.1.26.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-network-shuffle_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/lib/jackson-jaxrs-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/json-smart-1.3.1.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-api-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/commons-beanutils-1.7.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-streaming_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/lib/commons-io-2.4.jar":"System Classpath","/hadoop/lib/commons-beanutils-1.7.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/snappy-0.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jta-1.1.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-resourcemanager-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/protobuf-java-2.5.0.jar":"System Classpath","/hadoop-yarn-client/lib/curator-framework-2.7.1.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-daemon-1.0.13.jar":"System Classpath","/hadoop/lib/hadoop-lzo-0.6.0-sources.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-common.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/breeze_2.12-0.13.2.jar":"System Classpath","/hadoop/lib/jsp-api-2.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/orc-mapreduce-1.5.4-nohive.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-mapreduce-client-core-2.7.4.jar":"System Classpath","/hadoop/lib/htrace-core-3.1.0-incubating.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-encoding-1.10.0.jar":"System Classpath","/hadoop/hadoop-auth-2.7.3.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-applicationhistoryservice-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/jettison-1.1.jar":"System Classpath","/hadoop-yarn-client/lib/commons-codec-1.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-mapreduce-client-shuffle-2.7.4.jar":"System Classpath","/hadoop/lib/curator-recipes-2.7.1.jar":"System Classpath","/hadoop/lib/jetty-util-6.1.26.hwx.jar":"System Classpath","/hadoop-hdfs-client/lib/log4j-1.2.17.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/activation-1.1.1.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/jaxb-api-2.2.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/opencsv-2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-collections-3.2.2.jar":"System Classpath","/hadoop/lib/guava-11.0.2.jar":"System Classpath","/hadoop-yarn-client/lib/commons-net-3.1.jar":"System Classpath","/hadoop/lib/commons-beanutils-core-1.8.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/py4j-0.10.8.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/json4s-ast_2.12-3.5.3.jar":"System Classpath","/hadoop-yarn-client/lib/metrics-core-3.0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/pyrolite-4.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/chill_2.12-0.9.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/libfb303-0.9.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/javax.inject-1.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs.jar":"System Classpath","/ext/hadoop/*":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-web-proxy-2.7.3.jar":"System Classpath","/hadoop/lib/aws-java-sdk-core-1.10.6.jar":"System Classpath","/hadoop-yarn-client/lib/commons-configuration-1.6.jar":"System Classpath","/hadoop/lib/zookeeper-3.4.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-logging-1.1.3.jar":"System Classpath","/hadoop-hdfs-client/lib/jsr305-3.0.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/stringtemplate-3.2.1.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-resourcemanager.jar":"System Classpath","/hadoop/lib/netty-3.6.2.Final.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/zstd-jni-1.3.2-2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jsp-api-2.1.jar":"System Classpath","/hadoop-yarn-client/lib/jaxb-impl-2.2.3-1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-tags_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hive-beeline-1.2.1.spark2.jar":"System Classpath","/hadoop-hdfs-client/lib/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/snappy-java-1.0.4.1.jar":"System Classpath","/hadoop-yarn-client/lib/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs-nfs.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/hadoop/hadoop-aws-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/jsr305-3.0.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/datanucleus-rdbms-3.2.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/libthrift-0.9.3.jar":"System Classpath","/hadoop/lib/xz-1.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/json4s-core_2.12-3.5.3.jar":"System Classpath","/hadoop-yarn-client/lib/jersey-guice-1.9.jar":"System Classpath","/hadoop-yarn-client/lib/activation-1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-jaxrs-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/hdfs/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/mapreduce/lib/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-pool-1.5.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/arrow-memory-0.10.0.jar":"System Classpath","/hadoop-hdfs-client/lib/jetty-util-6.1.26.hwx.jar":"System Classpath","/hadoop/lib/ranger-plugin-classloader-0.7.0.jar":"System Classpath","/hadoop/lib/aws-java-sdk-kms-1.10.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-repl_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-yarn-client/lib/jersey-json-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-yarn_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-yarn-client/lib/xz-1.0.jar":"System Classpath","/hadoop/hadoop-common-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jetty-sslengine-6.1.26.jar":"System Classpath","/hadoop/lib/jaxb-api-2.2.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/metrics-graphite-3.1.5.jar":"System Classpath","/hadoop/lib/jcip-annotations-1.0-1.jar":"System Classpath","/hadoop/lib/nimbus-jose-jwt-4.41.1.jar":"System Classpath","/hadoop/lib/hadoop-lzo-0.6.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hk2-locator-2.4.0-b34.jar":"System Classpath","/hadoop/lib/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-logging-1.1.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jsr305-3.0.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-hive_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/RoaringBitmap-0.5.11.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/calcite-linq4j-1.2.0-incubating.jar":"System Classpath","/hadoop-yarn-client/lib/commons-beanutils-core-1.8.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-lang-2.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/derby-10.12.1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-hadoop-1.10.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/stax-api-1.0-2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/scala-reflect-2.12.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/calcite-core-1.2.0-incubating.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_conf__":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-format-2.4.0.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-sharedcachemanager-2.7.3.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-web-proxy.jar":"System Classpath","/hadoop-hdfs-client/lib/xercesImpl-2.9.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/netty-3.9.9.Final.jar":"System Classpath","/hadoop/lib/xmlenc-0.52.jar":"System Classpath","/hadoop/hadoop-annotations-2.7.3.jar":"System Classpath","/hadoop-hdfs-client/lib/protobuf-java-2.5.0.jar":"System Classpath","/hadoop-yarn-client/lib/jersey-core-1.9.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-registry-2.7.3.jar":"System Classpath","/hadoop-hdfs-client/lib/okhttp-2.7.5.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-yarn-common-2.7.4.jar":"System Classpath","/hadoop-yarn-client/lib/leveldbjni-all-1.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-crypto-1.0.0.jar":"System Classpath","/hadoop/lib/jetty-6.1.26.hwx.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/aopalliance-repackaged-2.4.0-b34.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_conf__/__hadoop_conf__":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-compress-1.8.1.jar":"System Classpath","/hadoop/lib/jackson-xc-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/gson-2.2.4.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-api.jar":"System Classpath","/hadoop/lib/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs-tests.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-io-2.4.jar":"System Classpath","/hadoop/hadoop-nfs.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-yarn-api-2.7.4.jar":"System Classpath","/hadoop/lib/json-smart-1.3.1.jar":"System Classpath","/hadoop-yarn-client/lib/commons-lang-2.6.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/azure-keyvault-core-0.8.0.jar":"System Classpath","/hadoop-yarn-client/lib/fst-2.24.jar":"System Classpath","/hadoop-yarn-client/lib/htrace-core-3.1.0-incubating.jar":"System Classpath","/hadoop/hadoop-azure-datalake.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-common-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hive-exec-1.2.1.spark2.jar":"System Classpath","/hadoop-hdfs-client/lib/jersey-server-1.9.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-sharedcachemanager.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-launcher_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/datanucleus-core-3.2.10.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/zookeeper-3.4.6.jar":"System Classpath","/hadoop-yarn-client/lib/commons-math3-3.1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/api-util-1.0.0-M20.jar":"System Classpath","/hadoop/lib/azure-keyvault-core-0.8.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-common-2.22.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jtransforms-2.4.0.jar":"System Classpath","/hadoop/lib/asm-3.2.jar":"System Classpath","/hadoop/lib/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/javax.ws.rs-api-2.0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/xmlenc-0.52.jar":"System Classpath","/hadoop/lib/jsch-0.1.54.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-timeline-pluginstorage.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-annotations-2.2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/hdfs/lib/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/httpclient-4.5.6.jar":"System Classpath","/hadoop-hdfs-client/lib/xmlenc-0.52.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-annotations-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/antlr-runtime-3.4.jar":"System Classpath","/etc/hadoop/conf/secure":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/avro-ipc-1.8.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/janino-3.0.11.jar":"System Classpath","/hadoop/lib/api-util-1.0.0-M20.jar":"System Classpath","/hadoop-yarn-client/lib/xmlenc-0.52.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/slf4j-api-1.7.16.jar":"System Classpath","/hadoop/lib/ranger-hdfs-plugin-shim-0.7.0.jar":"System Classpath","/hadoop-hdfs-client/lib/guava-11.0.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/chill-java-0.9.3.jar":"System Classpath","/hadoop-yarn-client/lib/aopalliance-1.0.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-applications-unmanaged-am-launcher.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/htrace-core-3.1.0-incubating.jar":"System Classpath","/hadoop/lib/jersey-json-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/guava-14.0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-yarn-server-web-proxy-2.7.4.jar":"System Classpath","/hadoop/lib/jetty-sslengine-6.1.26.hwx.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/ST4-4.0.4.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-client-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-sketch_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-yarn-client/lib/httpclient-4.5.2.jar":"System Classpath","/hadoop/lib/httpclient-4.5.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/antlr4-runtime-4.7.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/arrow-vector-0.10.0.jar":"System Classpath","/hadoop-yarn-client/lib/jersey-server-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-yarn-client-2.7.4.jar":"System Classpath","/hadoop-yarn-client/lib/jsch-0.1.54.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/curator-client-2.7.1.jar":"System Classpath","/hadoop-yarn-client/lib/stax-api-1.0-2.jar":"System Classpath","/hadoop-yarn-client/lib/commons-cli-1.2.jar":"System Classpath","/hadoop/lib/servlet-api-2.5.jar":"System Classpath","/hadoop-yarn-client/lib/commons-collections-3.2.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-net-3.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-media-jaxb-2.22.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-mapreduce-client-app-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-core_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-catalyst_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-kvstore_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/lib/joda-time-2.9.4.jar":"System Classpath","/hadoop-yarn-client/lib/jersey-client-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-lang3-3.8.1.jar":"System Classpath","/hadoop/lib/slf4j-log4j12-1.7.10.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/bonecp-0.8.0.RELEASE.jar":"System Classpath","/hadoop-yarn-client/lib/jetty-sslengine-6.1.26.hwx.jar":"System Classpath","/hadoop-yarn-client/lib/commons-logging-1.1.3.jar":"System Classpath","/hadoop/lib/jets3t-0.9.0.jar":"System Classpath","/hadoop/lib/hadoop-lzo-0.6.0-javadoc.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/orc-shims-1.5.4.jar":"System Classpath","/hadoop-hdfs-client/lib/okio-1.6.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-column-1.10.0.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-core-asl-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/jsp-api-2.1.jar":"System Classpath","/hadoop/lib/aws-java-sdk-s3-1.10.6.jar":"System Classpath","/hadoop/azure-data-lake-store-sdk-2.2.5.jar":"System Classpath","/hadoop-yarn-client/lib/zookeeper-3.4.6-tests.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/stream-2.7.0.jar":"System Classpath","/hadoop/hadoop-azure-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/jcip-annotations-1.0-1.jar":"System Classpath","/hadoop-yarn-client/lib/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/common/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/aircompressor-0.10.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/httpcore-4.4.10.jar":"System Classpath","/hadoop-yarn-client/lib/objenesis-2.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/avro-1.8.2.jar":"System Classpath","/hadoop-yarn-client/lib/jets3t-0.9.0.jar":"System Classpath","/hadoop-hdfs-client/lib/jackson-core-2.2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-configuration-1.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-mapreduce-client-jobclient-2.7.4.jar":"System Classpath","/hadoop-yarn-client/lib/commons-io-2.4.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-tests.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/guice-3.0.jar":"System Classpath","/hadoop/lib/jsr305-3.0.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/javassist-3.18.1-GA.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/curator-framework-2.7.1.jar":"System Classpath","/hadoop-yarn-client/lib/avro-1.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/scala-compiler-2.12.8.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-registry.jar":"System Classpath","/hadoop/lib/stax-api-1.0-2.jar":"System Classpath","/hadoop-yarn-client/lib/guice-servlet-3.0.jar":"System Classpath","/hadoop/lib/commons-math3-3.1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/javax.annotation-api-1.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jdo-api-3.0.1.jar":"System Classpath","/hadoop-hdfs-client/lib/jackson-core-asl-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-sql_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-yarn-server-common-2.7.4.jar":"System Classpath","/hadoop-yarn-client/lib/api-util-1.0.0-M20.jar":"System Classpath","/hadoop/lib/api-asn1-api-1.0.0-M20.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/minlog-1.3.0.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-xc-1.9.13.jar":"System Classpath","/hadoop-hdfs-client/lib/netty-3.6.2.Final.jar":"System Classpath","/hadoop/lib/hamcrest-core-1.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/xbean-asm7-shaded-4.12.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-core-2.2.3.jar":"System Classpath","/hadoop/lib/jaxb-impl-2.2.3-1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/scala-parser-combinators_2.12-1.1.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-io-2.4.jar":"System Classpath","/hadoop/lib/commons-lang-2.6.jar":"System Classpath","/hadoop/lib/commons-lang3-3.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spire_2.12-0.13.0.jar":"System Classpath","/hadoop-hdfs-client/lib/jackson-annotations-2.2.3.jar":"System Classpath","/hadoop/lib/jackson-core-asl-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/javax.servlet-api-3.1.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/core-1.1.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-cli-1.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/curator-recipes-2.7.1.jar":"System Classpath","/hadoop/lib/jersey-server-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spire-macros_2.12-0.13.0.jar":"System Classpath","/hadoop-yarn-client/lib/javassist-3.18.1-GA.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/validation-api-1.1.0.Final.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/protobuf-java-2.5.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-beanutils-core-1.8.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/gson-2.2.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/ivy-2.4.0.jar":"System Classpath","/hadoop-yarn-client/lib/paranamer-2.3.jar":"System Classpath","/hadoop/lib/gson-2.2.4.jar":"System Classpath","/hadoop/lib/jackson-annotations-2.2.3.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-databind-2.2.3.jar":"System Classpath","/hadoop-hdfs-client/lib/leveldbjni-all-1.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/guice-servlet-3.0.jar":"System Classpath","/hadoop/hadoop-annotations.jar":"System Classpath","/hadoop-yarn-client/lib/curator-recipes-2.7.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/flatbuffers-1.2.0-3f79e055.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-mllib_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-tests-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-common-1.10.0.jar":"System Classpath","/hadoop-yarn-client/lib/asm-3.2.jar":"System Classpath","/hadoop/lib/protobuf-java-2.5.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-core-2.9.6.jar":"System Classpath","/hadoop/lib/commons-configuration-1.6.jar":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Spark SQL basic example","App ID":"application_1547723113049_0006","Timestamp":1547788454200,"User":"spark","App Attempt ID":"1","Driver Logs":{"stdout":"http://node2:8042/node/containerlogs/container_e01_1547723113049_0005_01_000001/spark/stdout?start=-4096","stderr":"http://node2:8042/node/containerlogs/container_e01_1547723113049_0005_01_000001/spark/stderr?start=-4096"}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1547788454209,"Executor ID":"1","Executor Info":{"Host":"node3","Total Cores":1,"Log Urls":{"stdout":"http://node3:8042/node/containerlogs/container_e01_1547723113049_0005_01_000002/spark/stdout?start=-4096","stderr":"http://node3:8042/node/containerlogs/container_e01_1547723113049_0005_01_000002/spark/stderr?start=-4096"},"Attributes":{"NODE_HTTP_ADDRESS":"node3:8042","USER":"spark","LOG_FILES":"stderr,stdout","CLUSTER_ID":"yarn-cluster","HTTP_SCHEME":"http://","CONTAINER_ID":"container_e01_1547723113049_0005_01_000002"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"node3","Port":35087},"Maximum Memory":384093388,"Timestamp":1547788454413,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"Event":"SparkListenerExecutorAdded","Timestamp":1547788455343,"Executor ID":"2","Executor Info":{"Host":"node4","Total Cores":1,"Log Urls":{"stdout":"http://node4:8042/node/containerlogs/container_e01_1547723113049_0005_01_000003/spark/stdout?start=-4096","stderr":"http://node4:8042/node/containerlogs/container_e01_1547723113049_0005_01_000003/spark/stderr?start=-4096"},"Attributes":{"NODE_HTTP_ADDRESS":"node4:8042","USER":"spark","LOG_FILES":"stderr,stdout","CLUSTER_ID":"yarn-cluster","HTTP_SCHEME":"http://","CONTAINER_ID":"container_e01_1547723113049_0005_01_000003"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"2","Host":"node4","Port":44875},"Maximum Memory":384093388,"Timestamp":1547788455624,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1547788459293,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"json at SparkSQLExample.scala:59","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"mapPartitions\"}","Callsite":"json at SparkSQLExample.scala:59","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"json at SparkSQLExample.scala:59","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"json at SparkSQLExample.scala:59","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameReader.json(DataFrameReader.scala:331)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:59)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[0],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0006","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0006","spark.app.id":"application_1547723113049_0006","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"json at SparkSQLExample.scala:59","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"mapPartitions\"}","Callsite":"json at SparkSQLExample.scala:59","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"json at SparkSQLExample.scala:59","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"json at SparkSQLExample.scala:59","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameReader.json(DataFrameReader.scala:331)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:59)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788459321,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0006","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0006","spark.app.id":"application_1547723113049_0006","spark.hadoop.yarn.timeline-service.enabled":"false"}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1547788459627,"Executor ID":"1","Host":"node3","Locality":"NODE_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":1547788459627,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788463652,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"duration total (min, med, max)","Update":"1582","Value":"1581","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":0,"Name":"number of output rows","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"internal.metrics.input.recordsRead","Update":3,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":26,"Name":"internal.metrics.input.bytesRead","Update":73,"Value":73,"Internal":true,"Count Failed Values":true},{"ID":11,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.jvmGCTime","Update":159,"Value":159,"Internal":true,"Count Failed Values":true},{"ID":9,"Name":"internal.metrics.resultSize","Update":1963,"Value":1963,"Internal":true,"Count Failed Values":true},{"ID":8,"Name":"internal.metrics.executorCpuTime","Update":2225128499,"Value":2225128499,"Internal":true,"Count Failed Values":true},{"ID":7,"Name":"internal.metrics.executorRunTime","Update":2291,"Value":2291,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1171850277,"Value":1171850277,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.executorDeserializeTime","Update":1431,"Value":1431,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1431,"Executor Deserialize CPU Time":1171850277,"Executor Run Time":2291,"Executor CPU Time":2225128499,"Result Size":1963,"JVM GC Time":159,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":73,"Records Read":3},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} 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 f3e7d761dadeb..f4d3acc2acf7d 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 @@ -189,6 +189,34 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers } } + val casesCustomLogUrl = Seq( + "completed app executor list json apply custom log urls" -> + "applications/application_1547723113049_0005/1/executors", + "incomplete app executor list json apply custom log urls" -> + "applications/application_1547723113049_0006/1/executors", + ) + + casesCustomLogUrl.foreach { case (name, path) => + test(name) { + // This only verifies whether applying custom log URLs is in effect for SHS. + // Validation of "custom log URLs" functionality will be covered from different UTs. + + // restart server + try { + stop() + init((History.CUSTOM_EXECUTOR_LOG_URL.key, "http://newhost:9999/logs/clusters/" + + "{{CLUSTER_ID}}/users/{{USER}}/containers/{{CONTAINER_ID}}/{{FILE_NAME}})")) + + val expectationFile = new File(expRoot, HistoryServerSuite.sanitizePath(name) + + "_expectation.json") + assertApiCallResponse(path, expectationFile) + } finally { + // make sure other UTs are not affected from relaunching HistoryServer + stop() + } + } + } + // SPARK-10873 added the lastUpdated field for each application's attempt, // the REST API returns the last modified time of EVENT LOG file for this field. // It is not applicable to hard-code this dynamic field in a static expected file, @@ -632,26 +660,6 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers } } - test("Apply custom log urls") { - // This only verifies whether applying custom log URLs is in effect for SHS. - // Validation of "custom log URLs" functionality will be covered from different UTs. - - // restart server - try { - stop() - init((History.CUSTOM_EXECUTOR_LOG_URL.key, "http://newhost:9999/logs/clusters/" + - "{{CLUSTER_ID}}/users/{{USER}}/containers/{{CONTAINER_ID}}/{{FILE_NAME}})")) - - val path = "applications/application_1547723113049_0005/1/executors" - val expectation = "executor_list_json_apply_custom_log_urls_expectation.json" - - assertApiCallResponse(path, new File(expRoot, expectation)) - } finally { - // make sure other UTs are not affected from relaunching HistoryServer - stop() - } - } - def getContentAndCode(path: String, port: Int = port): (Int, Option[String], Option[String]) = { HistoryServerSuite.getContentAndCode(new URL(s"http://localhost:$port/api/v1/$path")) } diff --git a/dev/.rat-excludes b/dev/.rat-excludes index 93f86b1ee390f..9da5d1a1c8923 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -84,6 +84,7 @@ application_1516285256255_0012 application_1506645932520_24630151 application_1538416563558_0014 application_1547723113049_0005 +application_1547723113049_0006 stat local-1422981759269 local-1422981780767 From ca6be5f73c39fd152bf7e64e6bf26a32ad594d3e Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Sat, 19 Jan 2019 09:01:48 +0900 Subject: [PATCH 14/28] Also introduce a new config to toggle applying custom log url for incomplete apps --- .../history/HistoryAppStatusStore.scala | 31 ++++- .../spark/internal/config/History.scala | 6 + ...ply_incomplete_app_false_expectation.json} | 0 ...pply_incomplete_app_true_expectation.json} | 0 ...pply_incomplete_app_false_expectation.json | 114 ++++++++++++++++ ...apply_incomplete_app_true_expectation.json | 126 ++++++++++++++++++ .../deploy/history/HistoryServerSuite.scala | 39 +++--- docs/monitoring.md | 9 ++ 8 files changed, 304 insertions(+), 21 deletions(-) rename core/src/test/resources/HistoryServerExpectations/{completed_app_executor_list_json_apply_custom_log_urls_expectation.json => completed_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_false_expectation.json} (100%) rename core/src/test/resources/HistoryServerExpectations/{incomplete_app_executor_list_json_apply_custom_log_urls_expectation.json => completed_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_true_expectation.json} (100%) create mode 100644 core/src/test/resources/HistoryServerExpectations/incomplete_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_false_expectation.json create mode 100644 core/src/test/resources/HistoryServerExpectations/incomplete_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_true_expectation.json diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala index 86e0bfa9cc0c1..2b4210c7007b0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala @@ -21,7 +21,7 @@ import scala.util.matching.Regex import org.apache.spark.SparkConf import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.History.CUSTOM_EXECUTOR_LOG_URL +import org.apache.spark.internal.config.History.{APPLY_CUSTOM_EXECUTOR_LOG_URL_TO_INCOMPLETE_APP, CUSTOM_EXECUTOR_LOG_URL} import org.apache.spark.status.{AppStatusListener, AppStatusStore} import org.apache.spark.status.api.v1 import org.apache.spark.status.api.v1.ExecutorSummary @@ -36,21 +36,42 @@ private[spark] class HistoryAppStatusStore( import HistoryAppStatusStore._ private val logUrlPattern: Option[String] = conf.get(CUSTOM_EXECUTOR_LOG_URL) + private val applyReplaceLogUrlToIncompleteApp: Boolean = + conf.get(APPLY_CUSTOM_EXECUTOR_LOG_URL_TO_INCOMPLETE_APP) override def executorList(activeOnly: Boolean): Seq[v1.ExecutorSummary] = { + val execList = super.executorList(activeOnly) logUrlPattern match { - case Some(pattern) => super.executorList(activeOnly).map(replaceLogUrls(_, pattern)) - case None => super.executorList(activeOnly) + case Some(pattern) => + if (applyReplaceLogUrlToIncompleteApp || isApplicationCompleted) { + execList.map(replaceLogUrls(_, pattern)) + } else { + execList + } + + case None => execList } } override def executorSummary(executorId: String): v1.ExecutorSummary = { + val execSummary = super.executorSummary(executorId) logUrlPattern match { - case Some(pattern) => replaceLogUrls(super.executorSummary(executorId), pattern) - case None => super.executorSummary(executorId) + case Some(pattern) => + if (applyReplaceLogUrlToIncompleteApp || isApplicationCompleted) { + replaceLogUrls(execSummary, pattern) + } else { + execSummary + } + + case None => execSummary } } + private def isApplicationCompleted: Boolean = { + val appInfo = super.applicationInfo() + appInfo.attempts.nonEmpty && appInfo.attempts.head.completed + } + private def replaceLogUrls(exec: ExecutorSummary, urlPattern: String): ExecutorSummary = { val attributes = exec.attributes diff --git a/core/src/main/scala/org/apache/spark/internal/config/History.scala b/core/src/main/scala/org/apache/spark/internal/config/History.scala index 05ed848e4668d..7fa3a2954c13e 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/History.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/History.scala @@ -133,4 +133,10 @@ private[spark] object History { "resource manager to refer which patterns are supported.") .stringConf .createOptional + + val APPLY_CUSTOM_EXECUTOR_LOG_URL_TO_INCOMPLETE_APP = + ConfigBuilder("spark.history.custom.executor.log.url.applyIncompleteApplication") + .doc("Whether apply custom executor log url to incomplete application as well") + .booleanConf + .createWithDefault(true) } diff --git a/core/src/test/resources/HistoryServerExpectations/completed_app_executor_list_json_apply_custom_log_urls_expectation.json b/core/src/test/resources/HistoryServerExpectations/completed_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_false_expectation.json similarity index 100% rename from core/src/test/resources/HistoryServerExpectations/completed_app_executor_list_json_apply_custom_log_urls_expectation.json rename to core/src/test/resources/HistoryServerExpectations/completed_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_false_expectation.json diff --git a/core/src/test/resources/HistoryServerExpectations/incomplete_app_executor_list_json_apply_custom_log_urls_expectation.json b/core/src/test/resources/HistoryServerExpectations/completed_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_true_expectation.json similarity index 100% rename from core/src/test/resources/HistoryServerExpectations/incomplete_app_executor_list_json_apply_custom_log_urls_expectation.json rename to core/src/test/resources/HistoryServerExpectations/completed_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_true_expectation.json diff --git a/core/src/test/resources/HistoryServerExpectations/incomplete_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_false_expectation.json b/core/src/test/resources/HistoryServerExpectations/incomplete_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_false_expectation.json new file mode 100644 index 0000000000000..e558e6f2068c0 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/incomplete_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_false_expectation.json @@ -0,0 +1,114 @@ +[ { + "id" : "driver", + "hostPort" : "node2:40675", + "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" : 428762726, + "addTime" : "2019-01-18T05:14:05.481GMT", + "executorLogs" : { + "stdout" : "http://node2:8042/node/containerlogs/container_e01_1547723113049_0005_01_000001/spark/stdout?start=-4096", + "stderr" : "http://node2:8042/node/containerlogs/container_e01_1547723113049_0005_01_000001/spark/stderr?start=-4096" + }, + "memoryMetrics" : { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 428762726, + "totalOffHeapStorageMemory" : 0 + }, + "blacklistedInStages" : [ ], + "attributes" : { } +}, { + "id" : "2", + "hostPort" : "node4:44875", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 1, + "maxTasks" : 1, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 0, + "totalTasks" : 0, + "totalDuration" : 0, + "totalGCTime" : 0, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : false, + "maxMemory" : 384093388, + "addTime" : "2019-01-18T05:14:15.343GMT", + "executorLogs" : { + "stdout" : "http://node4:8042/node/containerlogs/container_e01_1547723113049_0005_01_000003/spark/stdout?start=-4096", + "stderr" : "http://node4:8042/node/containerlogs/container_e01_1547723113049_0005_01_000003/spark/stderr?start=-4096" + }, + "memoryMetrics" : { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 384093388, + "totalOffHeapStorageMemory" : 0 + }, + "blacklistedInStages" : [ ], + "attributes" : { + "NODE_HTTP_ADDRESS" : "node4:8042", + "USER" : "spark", + "LOG_FILES" : "stderr,stdout", + "CLUSTER_ID" : "yarn-cluster", + "HTTP_SCHEME" : "http://", + "CONTAINER_ID" : "container_e01_1547723113049_0005_01_000003" + } +}, { + "id" : "1", + "hostPort" : "node3:35087", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 1, + "maxTasks" : 1, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 1, + "totalTasks" : 1, + "totalDuration" : 4025, + "totalGCTime" : 159, + "totalInputBytes" : 73, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : false, + "maxMemory" : 384093388, + "addTime" : "2019-01-18T05:14:14.209GMT", + "executorLogs" : { + "stdout" : "http://node3:8042/node/containerlogs/container_e01_1547723113049_0005_01_000002/spark/stdout?start=-4096", + "stderr" : "http://node3:8042/node/containerlogs/container_e01_1547723113049_0005_01_000002/spark/stderr?start=-4096" + }, + "memoryMetrics" : { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 384093388, + "totalOffHeapStorageMemory" : 0 + }, + "blacklistedInStages" : [ ], + "attributes" : { + "NODE_HTTP_ADDRESS" : "node3:8042", + "USER" : "spark", + "LOG_FILES" : "stderr,stdout", + "CLUSTER_ID" : "yarn-cluster", + "HTTP_SCHEME" : "http://", + "CONTAINER_ID" : "container_e01_1547723113049_0005_01_000002" + } +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/incomplete_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_true_expectation.json b/core/src/test/resources/HistoryServerExpectations/incomplete_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_true_expectation.json new file mode 100644 index 0000000000000..4494767a45a63 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/incomplete_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_true_expectation.json @@ -0,0 +1,126 @@ +[ + { + "id":"driver", + "hostPort":"node2:40675", + "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":428762726, + "addTime":"2019-01-18T05:14:05.481GMT", + "executorLogs":{ + "stdout":"http://node2:8042/node/containerlogs/container_e01_1547723113049_0005_01_000001/spark/stdout?start=-4096", + "stderr":"http://node2:8042/node/containerlogs/container_e01_1547723113049_0005_01_000001/spark/stderr?start=-4096" + }, + "memoryMetrics":{ + "usedOnHeapStorageMemory":0, + "usedOffHeapStorageMemory":0, + "totalOnHeapStorageMemory":428762726, + "totalOffHeapStorageMemory":0 + }, + "blacklistedInStages":[ + + ], + "attributes":{ + + } + }, + { + "id":"2", + "hostPort":"node4:44875", + "isActive":true, + "rddBlocks":0, + "memoryUsed":0, + "diskUsed":0, + "totalCores":1, + "maxTasks":1, + "activeTasks":0, + "failedTasks":0, + "completedTasks":0, + "totalTasks":0, + "totalDuration":0, + "totalGCTime":0, + "totalInputBytes":0, + "totalShuffleRead":0, + "totalShuffleWrite":0, + "isBlacklisted":false, + "maxMemory":384093388, + "addTime":"2019-01-18T05:14:15.343GMT", + "executorLogs":{ + "stderr":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000003/stderr)", + "stdout":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000003/stdout)" + }, + "memoryMetrics":{ + "usedOnHeapStorageMemory":0, + "usedOffHeapStorageMemory":0, + "totalOnHeapStorageMemory":384093388, + "totalOffHeapStorageMemory":0 + }, + "blacklistedInStages":[ + + ], + "attributes":{ + "NODE_HTTP_ADDRESS":"node4:8042", + "USER":"spark", + "LOG_FILES":"stderr,stdout", + "CLUSTER_ID":"yarn-cluster", + "HTTP_SCHEME":"http://", + "CONTAINER_ID":"container_e01_1547723113049_0005_01_000003" + } + }, + { + "id":"1", + "hostPort":"node3:35087", + "isActive":true, + "rddBlocks":0, + "memoryUsed":0, + "diskUsed":0, + "totalCores":1, + "maxTasks":1, + "activeTasks":0, + "failedTasks":0, + "completedTasks":1, + "totalTasks":1, + "totalDuration":4025, + "totalGCTime":159, + "totalInputBytes":73, + "totalShuffleRead":0, + "totalShuffleWrite":0, + "isBlacklisted":false, + "maxMemory":384093388, + "addTime":"2019-01-18T05:14:14.209GMT", + "executorLogs":{ + "stderr":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000002/stderr)", + "stdout":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000002/stdout)" + }, + "memoryMetrics":{ + "usedOnHeapStorageMemory":0, + "usedOffHeapStorageMemory":0, + "totalOnHeapStorageMemory":384093388, + "totalOffHeapStorageMemory":0 + }, + "blacklistedInStages":[ + + ], + "attributes":{ + "NODE_HTTP_ADDRESS":"node3:8042", + "USER":"spark", + "LOG_FILES":"stderr,stdout", + "CLUSTER_ID":"yarn-cluster", + "HTTP_SCHEME":"http://", + "CONTAINER_ID":"container_e01_1547723113049_0005_01_000002" + } + } +] \ No newline at end of file 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 f4d3acc2acf7d..897d3e462fcde 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 @@ -197,22 +197,29 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers ) casesCustomLogUrl.foreach { case (name, path) => - test(name) { - // This only verifies whether applying custom log URLs is in effect for SHS. - // Validation of "custom log URLs" functionality will be covered from different UTs. - - // restart server - try { - stop() - init((History.CUSTOM_EXECUTOR_LOG_URL.key, "http://newhost:9999/logs/clusters/" + - "{{CLUSTER_ID}}/users/{{USER}}/containers/{{CONTAINER_ID}}/{{FILE_NAME}})")) - - val expectationFile = new File(expRoot, HistoryServerSuite.sanitizePath(name) + - "_expectation.json") - assertApiCallResponse(path, expectationFile) - } finally { - // make sure other UTs are not affected from relaunching HistoryServer - stop() + Seq(true, false).foreach { applyToIncompleteApp => + val newName = name + s" apply incomplete app $applyToIncompleteApp" + test(newName) { + // This only verifies whether applying custom log URLs is in effect for SHS. + // Validation of "custom log URLs" functionality will be covered from different UTs. + + // restart server + try { + stop() + init( + // compiler complains when using `->`, so made a tuple + (CUSTOM_EXECUTOR_LOG_URL.key, "http://newhost:9999/logs/clusters/" + + "{{CLUSTER_ID}}/users/{{USER}}/containers/{{CONTAINER_ID}}/{{FILE_NAME}})"), + APPLY_CUSTOM_EXECUTOR_LOG_URL_TO_INCOMPLETE_APP.key -> applyToIncompleteApp.toString + ) + + val expectationFile = new File(expRoot, HistoryServerSuite.sanitizePath(newName) + + "_expectation.json") + assertApiCallResponse(path, expectationFile) + } finally { + // make sure other UTs are not affected from relaunching HistoryServer + stop() + } } } } diff --git a/docs/monitoring.md b/docs/monitoring.md index f79f5de379e4a..4617d9d6c5d0d 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -259,6 +259,15 @@ Security options for the Spark History Server are covered more detail in the For now, YARN mode only supports this configuration. + + spark.history.custom.executor.log.url.applyIncompleteApplication + false + + Specifies whether apply custom spark executor log url to incomplete applications as well. + If executor logs for running applications should be provided as origin log urls, set this to `false`. + Please note that incomplete applications may include applications which didn't shutdown gracefully. + + From 229a75a41f6d027aae4f26f8fa2fe04adebff690 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Sat, 19 Jan 2019 11:01:06 +0900 Subject: [PATCH 15/28] Also apply custom log url to driver log as well (in case of YARN cluster mode) --- .../scala/org/apache/spark/SparkContext.scala | 3 +- .../spark/scheduler/SchedulerBackend.scala | 7 ++++ .../spark/scheduler/SparkListener.scala | 3 +- .../spark/status/AppStatusListener.scala | 1 + .../org/apache/spark/util/JsonProtocol.scala | 7 +++- ...pply_incomplete_app_false_expectation.json | 21 ++++++---- ...apply_incomplete_app_true_expectation.json | 21 ++++++---- ...pply_incomplete_app_false_expectation.json | 9 ++++- ...apply_incomplete_app_true_expectation.json | 21 ++++++---- .../application_1547723113049_0005 | 2 +- .../application_1547723113049_0006 | 2 +- .../deploy/history/HistoryServerSuite.scala | 7 ++-- .../cluster/YarnClusterSchedulerBackend.scala | 40 +++++++++++++++++++ .../spark/deploy/yarn/YarnClusterSuite.scala | 35 ++++++++++++++++ 14 files changed, 144 insertions(+), 35 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index f2b79b84b63c8..6273601d9986b 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -2355,7 +2355,8 @@ class SparkContext(config: SparkConf) extends Logging { // Note: this code assumes that the task scheduler has been initialized and has contacted // the cluster manager to get an application ID (in case the cluster manager provides one). listenerBus.post(SparkListenerApplicationStart(appName, Some(applicationId), - startTime, sparkUser, applicationAttemptId, schedulerBackend.getDriverLogUrls)) + startTime, sparkUser, applicationAttemptId, schedulerBackend.getDriverLogUrls, + schedulerBackend.getDriverAttributes)) _driverLogger.foreach(_.startSync(_hadoopConfiguration)) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala index c187ee146301b..9159d2a0158d5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala @@ -69,6 +69,13 @@ private[spark] trait SchedulerBackend { */ def getDriverLogUrls: Option[Map[String, String]] = None + /** + * Get the attributes on driver. These attributes are used to replace log URLs when + * custom log url pattern is specified. + * @return Map containing attributes on driver. + */ + def getDriverAttributes: Option[Map[String, String]] = None + /** * Get the max number of tasks that can be concurrent launched currently. * Note that please don't cache the value returned by this method, because the number can change diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index e92b8a2718df0..b332c0f46f14d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -192,7 +192,8 @@ case class SparkListenerApplicationStart( time: Long, sparkUser: String, appAttemptId: Option[String], - driverLogs: Option[Map[String, String]] = None) extends SparkListenerEvent + driverLogs: Option[Map[String, String]] = None, + driverAttributes: Option[Map[String, String]] = None) extends SparkListenerEvent @DeveloperApi case class SparkListenerApplicationEnd(time: Long) extends SparkListenerEvent diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala index ce6e1f7b11b8e..f69c7dd2cb6d6 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -131,6 +131,7 @@ private[spark] class AppStatusListener( .orElse(liveExecutors.get(SparkContext.LEGACY_DRIVER_IDENTIFIER)) driver.foreach { d => d.executorLogs = logs.toMap + d.attributes = event.driverAttributes.getOrElse(Map.empty).toMap update(d, System.nanoTime()) } } diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 11dcf2b5bf88d..d84dd5800ebb7 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -211,7 +211,8 @@ private[spark] object JsonProtocol { ("Timestamp" -> applicationStart.time) ~ ("User" -> applicationStart.sparkUser) ~ ("App Attempt ID" -> applicationStart.appAttemptId.map(JString(_)).getOrElse(JNothing)) ~ - ("Driver Logs" -> applicationStart.driverLogs.map(mapToJson).getOrElse(JNothing)) + ("Driver Logs" -> applicationStart.driverLogs.map(mapToJson).getOrElse(JNothing)) ~ + ("Driver Attributes" -> applicationStart.driverAttributes.map(mapToJson).getOrElse(JNothing)) } def applicationEndToJson(applicationEnd: SparkListenerApplicationEnd): JValue = { @@ -694,7 +695,9 @@ private[spark] object JsonProtocol { val sparkUser = (json \ "User").extract[String] val appAttemptId = jsonOption(json \ "App Attempt ID").map(_.extract[String]) val driverLogs = jsonOption(json \ "Driver Logs").map(mapFromJson) - SparkListenerApplicationStart(appName, appId, time, sparkUser, appAttemptId, driverLogs) + val driverAttributes = jsonOption(json \ "Driver Attributes").map(mapFromJson) + SparkListenerApplicationStart(appName, appId, time, sparkUser, appAttemptId, driverLogs, + driverAttributes) } def applicationEndFromJson(json: JValue): SparkListenerApplicationEnd = { diff --git a/core/src/test/resources/HistoryServerExpectations/completed_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_false_expectation.json b/core/src/test/resources/HistoryServerExpectations/completed_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_false_expectation.json index 4494767a45a63..a7535680b4704 100644 --- a/core/src/test/resources/HistoryServerExpectations/completed_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_false_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/completed_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_false_expectation.json @@ -21,8 +21,8 @@ "maxMemory":428762726, "addTime":"2019-01-18T05:14:05.481GMT", "executorLogs":{ - "stdout":"http://node2:8042/node/containerlogs/container_e01_1547723113049_0005_01_000001/spark/stdout?start=-4096", - "stderr":"http://node2:8042/node/containerlogs/container_e01_1547723113049_0005_01_000001/spark/stderr?start=-4096" + "stderr" : "http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000001/stderr", + "stdout" : "http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000001/stdout" }, "memoryMetrics":{ "usedOnHeapStorageMemory":0, @@ -33,8 +33,13 @@ "blacklistedInStages":[ ], - "attributes":{ - + "attributes" : { + "NODE_HTTP_ADDRESS" : "node2:8042", + "USER" : "spark", + "LOG_FILES" : "stderr,stdout", + "CLUSTER_ID" : "yarn-cluster", + "HTTP_SCHEME" : "http://", + "CONTAINER_ID" : "container_e01_1547723113049_0005_01_000001" } }, { @@ -59,8 +64,8 @@ "maxMemory":384093388, "addTime":"2019-01-18T05:14:15.343GMT", "executorLogs":{ - "stderr":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000003/stderr)", - "stdout":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000003/stdout)" + "stderr":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000003/stderr", + "stdout":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000003/stdout" }, "memoryMetrics":{ "usedOnHeapStorageMemory":0, @@ -102,8 +107,8 @@ "maxMemory":384093388, "addTime":"2019-01-18T05:14:14.209GMT", "executorLogs":{ - "stderr":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000002/stderr)", - "stdout":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000002/stdout)" + "stderr":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000002/stderr", + "stdout":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000002/stdout" }, "memoryMetrics":{ "usedOnHeapStorageMemory":0, diff --git a/core/src/test/resources/HistoryServerExpectations/completed_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_true_expectation.json b/core/src/test/resources/HistoryServerExpectations/completed_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_true_expectation.json index 4494767a45a63..a7535680b4704 100644 --- a/core/src/test/resources/HistoryServerExpectations/completed_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_true_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/completed_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_true_expectation.json @@ -21,8 +21,8 @@ "maxMemory":428762726, "addTime":"2019-01-18T05:14:05.481GMT", "executorLogs":{ - "stdout":"http://node2:8042/node/containerlogs/container_e01_1547723113049_0005_01_000001/spark/stdout?start=-4096", - "stderr":"http://node2:8042/node/containerlogs/container_e01_1547723113049_0005_01_000001/spark/stderr?start=-4096" + "stderr" : "http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000001/stderr", + "stdout" : "http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000001/stdout" }, "memoryMetrics":{ "usedOnHeapStorageMemory":0, @@ -33,8 +33,13 @@ "blacklistedInStages":[ ], - "attributes":{ - + "attributes" : { + "NODE_HTTP_ADDRESS" : "node2:8042", + "USER" : "spark", + "LOG_FILES" : "stderr,stdout", + "CLUSTER_ID" : "yarn-cluster", + "HTTP_SCHEME" : "http://", + "CONTAINER_ID" : "container_e01_1547723113049_0005_01_000001" } }, { @@ -59,8 +64,8 @@ "maxMemory":384093388, "addTime":"2019-01-18T05:14:15.343GMT", "executorLogs":{ - "stderr":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000003/stderr)", - "stdout":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000003/stdout)" + "stderr":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000003/stderr", + "stdout":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000003/stdout" }, "memoryMetrics":{ "usedOnHeapStorageMemory":0, @@ -102,8 +107,8 @@ "maxMemory":384093388, "addTime":"2019-01-18T05:14:14.209GMT", "executorLogs":{ - "stderr":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000002/stderr)", - "stdout":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000002/stdout)" + "stderr":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000002/stderr", + "stdout":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000002/stdout" }, "memoryMetrics":{ "usedOnHeapStorageMemory":0, diff --git a/core/src/test/resources/HistoryServerExpectations/incomplete_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_false_expectation.json b/core/src/test/resources/HistoryServerExpectations/incomplete_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_false_expectation.json index e558e6f2068c0..d4bd46c36bc4f 100644 --- a/core/src/test/resources/HistoryServerExpectations/incomplete_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_false_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/incomplete_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_false_expectation.json @@ -30,7 +30,14 @@ "totalOffHeapStorageMemory" : 0 }, "blacklistedInStages" : [ ], - "attributes" : { } + "attributes" : { + "NODE_HTTP_ADDRESS" : "node2:8042", + "USER" : "spark", + "LOG_FILES" : "stderr,stdout", + "CLUSTER_ID" : "yarn-cluster", + "HTTP_SCHEME" : "http://", + "CONTAINER_ID" : "container_e01_1547723113049_0005_01_000001" + } }, { "id" : "2", "hostPort" : "node4:44875", diff --git a/core/src/test/resources/HistoryServerExpectations/incomplete_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_true_expectation.json b/core/src/test/resources/HistoryServerExpectations/incomplete_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_true_expectation.json index 4494767a45a63..a7535680b4704 100644 --- a/core/src/test/resources/HistoryServerExpectations/incomplete_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_true_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/incomplete_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_true_expectation.json @@ -21,8 +21,8 @@ "maxMemory":428762726, "addTime":"2019-01-18T05:14:05.481GMT", "executorLogs":{ - "stdout":"http://node2:8042/node/containerlogs/container_e01_1547723113049_0005_01_000001/spark/stdout?start=-4096", - "stderr":"http://node2:8042/node/containerlogs/container_e01_1547723113049_0005_01_000001/spark/stderr?start=-4096" + "stderr" : "http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000001/stderr", + "stdout" : "http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000001/stdout" }, "memoryMetrics":{ "usedOnHeapStorageMemory":0, @@ -33,8 +33,13 @@ "blacklistedInStages":[ ], - "attributes":{ - + "attributes" : { + "NODE_HTTP_ADDRESS" : "node2:8042", + "USER" : "spark", + "LOG_FILES" : "stderr,stdout", + "CLUSTER_ID" : "yarn-cluster", + "HTTP_SCHEME" : "http://", + "CONTAINER_ID" : "container_e01_1547723113049_0005_01_000001" } }, { @@ -59,8 +64,8 @@ "maxMemory":384093388, "addTime":"2019-01-18T05:14:15.343GMT", "executorLogs":{ - "stderr":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000003/stderr)", - "stdout":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000003/stdout)" + "stderr":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000003/stderr", + "stdout":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000003/stdout" }, "memoryMetrics":{ "usedOnHeapStorageMemory":0, @@ -102,8 +107,8 @@ "maxMemory":384093388, "addTime":"2019-01-18T05:14:14.209GMT", "executorLogs":{ - "stderr":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000002/stderr)", - "stdout":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000002/stdout)" + "stderr":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000002/stderr", + "stdout":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000002/stdout" }, "memoryMetrics":{ "usedOnHeapStorageMemory":0, diff --git a/core/src/test/resources/spark-events/application_1547723113049_0005 b/core/src/test/resources/spark-events/application_1547723113049_0005 index a409762608a47..41ca28d8ac436 100644 --- a/core/src/test/resources/spark-events/application_1547723113049_0005 +++ b/core/src/test/resources/spark-events/application_1547723113049_0005 @@ -1,7 +1,7 @@ {"Event":"SparkListenerLogStart","Spark Version":"3.0.0-SNAPSHOT"} {"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"node2","Port":40675},"Maximum Memory":428762726,"Timestamp":1547788445481,"Maximum Onheap Memory":428762726,"Maximum Offheap Memory":0} {"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre","Java Version":"1.8.0_161 (Oracle Corporation)","Scala Version":"version 2.12.8"},"Spark Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.scheduler.mode":"FIFO","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"},"System Properties":{"java.io.tmpdir":"/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_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/yarn","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/amd64","user.dir":"/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","java.library.path":"/usr/java/packages/lib/amd64:/usr/lib64:/lib64:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.161-b14","jetty.git.hash":"27208684755d94a92186989f695db2d7b21ebc51","java.endorsed.dirs":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/endorsed","java.runtime.version":"1.8.0_161-b14","java.vm.info":"mixed mode","java.ext.dirs":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"OpenJDK Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/resources.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/rt.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/sunrsasign.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/jsse.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/jce.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/charsets.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/jfr.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/classes","file.encoding":"UTF-8","user.timezone":"UTC","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"4.18.5-1.el7.elrepo.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":"yarn","hdp.version":"2.6.5.1002-73","java.vm.name":"OpenJDK 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.yarn.ApplicationMaster --class org.apache.spark.examples.sql.SparkSQLExample --jar file:/home/spark/spark-3.0.0-snapshot-spark-26311-v2/examples/jars/spark-examples_2.12-3.0.0-SNAPSHOT.jar --properties-file /hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_conf__/__spark_conf__.properties --dist-cache-conf /hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_conf__/__spark_dist_cache__.properties","java.home":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre","java.version":"1.8.0_161","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/hadoop-yarn-client/lib/commons-digester-1.8.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-applications-distributedshell-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-compiler-3.0.11.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-hadoop-bundle-1.6.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/lz4-java-1.5.0.jar":"System Classpath","/hadoop/lib/commons-digester-1.8.jar":"System Classpath","/hadoop-yarn-client/lib/netty-3.6.2.Final.jar":"System Classpath","/hadoop/lib/curator-client-2.7.1.jar":"System Classpath","/hadoop/lib/log4j-1.2.17.jar":"System Classpath","/hadoop/lib/paranamer-2.3.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-codec-1.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hk2-api-2.4.0-b34.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/orc-core-1.5.4-nohive.jar":"System Classpath","/hadoop-yarn-client/lib/curator-client-2.7.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop/hadoop-common.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-cli-1.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-hdfs-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/aopalliance-1.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-guava-2.22.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/stax-api-1.0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/snappy-java-1.1.7.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/univocity-parsers-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/azure-storage-5.4.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/netty-all-4.1.30.Final.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-container-servlet-core-2.22.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/macro-compat_2.12-1.1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-httpclient-3.1.jar":"System Classpath","/hadoop/lib/httpcore-4.4.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/hadoop/lib/java-xmlbuilder-0.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jodd-core-3.5.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/breeze-macros_2.12-0.13.2.jar":"System Classpath","/hadoop-yarn-client/lib/zookeeper-3.4.6.jar":"System Classpath","/hadoop/lib/curator-framework-2.7.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jul-to-slf4j-1.7.16.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/xz-1.5.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/antlr-2.7.7.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/json4s-scalap_2.12-3.5.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/apache-log4j-extras-1.2.17.jar":"System Classpath","/hadoop-hdfs-client/lib/htrace-core-3.1.0-incubating.jar":"System Classpath","/hadoop/hadoop-azure.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-nodemanager.jar":"System Classpath","/hadoop-yarn-client/lib/servlet-api-2.5.jar":"System Classpath","/hadoop-yarn-client/lib/guava-11.0.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/arpack_combined_all-0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/arrow-format-0.10.0.jar":"System Classpath","/hadoop-yarn-client/lib/guice-3.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/scala-library-2.12.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-container-servlet-2.22.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/xercesImpl-2.9.1.jar":"System Classpath","/hadoop/hadoop-azure-datalake-2.7.3.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-common.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/yarn/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-auth-2.7.4.jar":"System Classpath","/hadoop/lib/activation-1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jline-2.14.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/yarn/lib/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hive-metastore-1.2.1.spark2.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-applicationhistoryservice.jar":"System Classpath","/hadoop/lib/ojdbc6.jar":"System Classpath","/hadoop/lib/commons-collections-3.2.2.jar":"System Classpath","/hadoop-hdfs-client/lib/servlet-api-2.5.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/joda-time-2.9.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-annotations-2.9.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-mllib-local_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/leveldbjni-all-1.8.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-common-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/json4s-jackson_2.12-3.5.3.jar":"System Classpath","/hadoop/conf":"System Classpath","/hadoop/hadoop-common-2.7.3-tests.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-timeline-pluginstorage-2.7.3.jar":"System Classpath","/hadoop/lib/junit-4.11.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs-2.7.3-tests.jar":"System Classpath","/hadoop-yarn-client/lib/httpcore-4.4.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-unsafe_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/metrics-jvm-3.1.5.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/objenesis-2.5.1.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-nodemanager-2.7.3.jar":"System Classpath","/hadoop/hadoop-common-tests.jar":"System Classpath","/hadoop-hdfs-client/lib/jackson-databind-2.2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-graphx_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-hdfs-client/lib/asm-3.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/kryo-shaded-4.0.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/compress-lzf-1.0.3.jar":"System Classpath","/hadoop/lib/azure-storage-5.4.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/avro-mapred-1.8.2-hadoop2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/scala-xml_2.12-1.0.5.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-applications-distributedshell.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jaxb-api-2.2.2.jar":"System Classpath","/hadoop/lib/commons-net-3.1.jar":"System Classpath","/hadoop-yarn-client/lib/java-xmlbuilder-0.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/paranamer-2.8.jar":"System Classpath","/hadoop/lib/commons-logging-1.1.3.jar":"System Classpath","/hadoop-yarn-client/lib/log4j-1.2.17.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/mapreduce/*":"System Classpath","/hadoop/lib/commons-codec-1.4.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs-nfs-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/commons-lang3-3.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-math3-3.4.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-network-common_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jetty-6.1.26.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/javolution-5.5.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jcl-over-slf4j-1.7.16.jar":"System Classpath","/hadoop/lib/slf4j-api-1.7.10.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-common-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-core-asl-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hk2-utils-2.4.0-b34.jar":"System Classpath","/hadoop/lib/mockito-all-1.8.5.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-client.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/common/lib/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hppc-0.7.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/shapeless_2.12-2.3.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-mapreduce-client-common-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/log4j-1.2.17.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-module-scala_2.12-2.9.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-module-paranamer-2.9.6.jar":"System Classpath","/hadoop-hdfs-client/lib/jetty-6.1.26.hwx.jar":"System Classpath","/hadoop/hadoop-aws.jar":"System Classpath","/hadoop/lib/commons-cli-1.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-codec-1.10.jar":"System Classpath","/hadoop/lib/commons-compress-1.4.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-server-2.22.2.jar":"System Classpath","/hadoop/lib/avro-1.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-xc-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-dbcp-1.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-beanutils-1.7.0.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-jaxrs-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/api-asn1-api-1.0.0-M20.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-client-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/javax.inject-2.4.0-b34.jar":"System Classpath","/hadoop/gcs-connector-1.8.1-shaded.jar":"System Classpath","/hadoop-hdfs-client/lib/jersey-core-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/metrics-json-3.1.5.jar":"System Classpath","/hadoop/lib/jackson-core-2.2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jpam-1.1.jar":"System Classpath","/hadoop/lib/jersey-core-1.9.jar":"System Classpath","/hadoop/lib/snappy-java-1.0.4.1.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-lang-2.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/slf4j-log4j12-1.7.16.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/osgi-resource-locator-1.0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/api-asn1-api-1.0.0-M20.jar":"System Classpath","/hadoop-yarn-client/lib/nimbus-jose-jwt-4.41.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-databind-2.9.6.jar":"System Classpath","/hadoop-hdfs-client/lib/xml-apis-1.3.04.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-client-2.22.2.jar":"System Classpath","/hadoop/hadoop-nfs-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hive-jdbc-1.2.1.spark2.jar":"System Classpath","/hadoop/lib/jackson-databind-2.2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/eigenbase-properties-1.1.5.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/machinist_2.12-0.6.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/tools/lib/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-jackson-1.10.0.jar":"System Classpath","/hadoop-yarn-client/lib/jetty-util-6.1.26.hwx.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-digester-1.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/calcite-avatica-1.2.0-incubating.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/oro-2.0.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hive-cli-1.2.1.spark2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-hive-thriftserver_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-applications-unmanaged-am-launcher-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/JavaEWAH-0.3.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/super-csv-2.2.0.jar":"System Classpath","/hadoop/lib/ranger-yarn-plugin-shim-0.7.0.jar":"System Classpath","/hadoop-yarn-client/lib/jetty-6.1.26.hwx.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/metrics-core-3.1.5.jar":"System Classpath","/hadoop-yarn-client/lib/commons-compress-1.4.1.jar":"System Classpath","/hadoop-hdfs-client/lib/netty-all-4.0.52.Final.jar":"System Classpath","/hadoop/hadoop-auth.jar":"System Classpath","/hadoop-yarn-client/lib/javax.inject-1.jar":"System Classpath","/hadoop/lib/jettison-1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jetty-util-6.1.26.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-network-shuffle_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/lib/jackson-jaxrs-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/json-smart-1.3.1.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-api-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/commons-beanutils-1.7.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-streaming_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/lib/commons-io-2.4.jar":"System Classpath","/hadoop/lib/commons-beanutils-1.7.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/snappy-0.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jta-1.1.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-resourcemanager-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/protobuf-java-2.5.0.jar":"System Classpath","/hadoop-yarn-client/lib/curator-framework-2.7.1.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-daemon-1.0.13.jar":"System Classpath","/hadoop/lib/hadoop-lzo-0.6.0-sources.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-common.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/breeze_2.12-0.13.2.jar":"System Classpath","/hadoop/lib/jsp-api-2.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/orc-mapreduce-1.5.4-nohive.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-mapreduce-client-core-2.7.4.jar":"System Classpath","/hadoop/lib/htrace-core-3.1.0-incubating.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-encoding-1.10.0.jar":"System Classpath","/hadoop/hadoop-auth-2.7.3.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-applicationhistoryservice-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/jettison-1.1.jar":"System Classpath","/hadoop-yarn-client/lib/commons-codec-1.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-mapreduce-client-shuffle-2.7.4.jar":"System Classpath","/hadoop/lib/curator-recipes-2.7.1.jar":"System Classpath","/hadoop/lib/jetty-util-6.1.26.hwx.jar":"System Classpath","/hadoop-hdfs-client/lib/log4j-1.2.17.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/activation-1.1.1.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/jaxb-api-2.2.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/opencsv-2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-collections-3.2.2.jar":"System Classpath","/hadoop/lib/guava-11.0.2.jar":"System Classpath","/hadoop-yarn-client/lib/commons-net-3.1.jar":"System Classpath","/hadoop/lib/commons-beanutils-core-1.8.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/py4j-0.10.8.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/json4s-ast_2.12-3.5.3.jar":"System Classpath","/hadoop-yarn-client/lib/metrics-core-3.0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/pyrolite-4.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/chill_2.12-0.9.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/libfb303-0.9.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/javax.inject-1.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs.jar":"System Classpath","/ext/hadoop/*":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-web-proxy-2.7.3.jar":"System Classpath","/hadoop/lib/aws-java-sdk-core-1.10.6.jar":"System Classpath","/hadoop-yarn-client/lib/commons-configuration-1.6.jar":"System Classpath","/hadoop/lib/zookeeper-3.4.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-logging-1.1.3.jar":"System Classpath","/hadoop-hdfs-client/lib/jsr305-3.0.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/stringtemplate-3.2.1.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-resourcemanager.jar":"System Classpath","/hadoop/lib/netty-3.6.2.Final.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/zstd-jni-1.3.2-2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jsp-api-2.1.jar":"System Classpath","/hadoop-yarn-client/lib/jaxb-impl-2.2.3-1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-tags_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hive-beeline-1.2.1.spark2.jar":"System Classpath","/hadoop-hdfs-client/lib/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/snappy-java-1.0.4.1.jar":"System Classpath","/hadoop-yarn-client/lib/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs-nfs.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/hadoop/hadoop-aws-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/jsr305-3.0.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/datanucleus-rdbms-3.2.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/libthrift-0.9.3.jar":"System Classpath","/hadoop/lib/xz-1.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/json4s-core_2.12-3.5.3.jar":"System Classpath","/hadoop-yarn-client/lib/jersey-guice-1.9.jar":"System Classpath","/hadoop-yarn-client/lib/activation-1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-jaxrs-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/hdfs/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/mapreduce/lib/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-pool-1.5.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/arrow-memory-0.10.0.jar":"System Classpath","/hadoop-hdfs-client/lib/jetty-util-6.1.26.hwx.jar":"System Classpath","/hadoop/lib/ranger-plugin-classloader-0.7.0.jar":"System Classpath","/hadoop/lib/aws-java-sdk-kms-1.10.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-repl_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-yarn-client/lib/jersey-json-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-yarn_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-yarn-client/lib/xz-1.0.jar":"System Classpath","/hadoop/hadoop-common-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jetty-sslengine-6.1.26.jar":"System Classpath","/hadoop/lib/jaxb-api-2.2.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/metrics-graphite-3.1.5.jar":"System Classpath","/hadoop/lib/jcip-annotations-1.0-1.jar":"System Classpath","/hadoop/lib/nimbus-jose-jwt-4.41.1.jar":"System Classpath","/hadoop/lib/hadoop-lzo-0.6.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hk2-locator-2.4.0-b34.jar":"System Classpath","/hadoop/lib/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-logging-1.1.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jsr305-3.0.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-hive_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/RoaringBitmap-0.5.11.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/calcite-linq4j-1.2.0-incubating.jar":"System Classpath","/hadoop-yarn-client/lib/commons-beanutils-core-1.8.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-lang-2.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/derby-10.12.1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-hadoop-1.10.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/stax-api-1.0-2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/scala-reflect-2.12.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/calcite-core-1.2.0-incubating.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_conf__":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-format-2.4.0.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-sharedcachemanager-2.7.3.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-web-proxy.jar":"System Classpath","/hadoop-hdfs-client/lib/xercesImpl-2.9.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/netty-3.9.9.Final.jar":"System Classpath","/hadoop/lib/xmlenc-0.52.jar":"System Classpath","/hadoop/hadoop-annotations-2.7.3.jar":"System Classpath","/hadoop-hdfs-client/lib/protobuf-java-2.5.0.jar":"System Classpath","/hadoop-yarn-client/lib/jersey-core-1.9.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-registry-2.7.3.jar":"System Classpath","/hadoop-hdfs-client/lib/okhttp-2.7.5.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-yarn-common-2.7.4.jar":"System Classpath","/hadoop-yarn-client/lib/leveldbjni-all-1.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-crypto-1.0.0.jar":"System Classpath","/hadoop/lib/jetty-6.1.26.hwx.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/aopalliance-repackaged-2.4.0-b34.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_conf__/__hadoop_conf__":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-compress-1.8.1.jar":"System Classpath","/hadoop/lib/jackson-xc-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/gson-2.2.4.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-api.jar":"System Classpath","/hadoop/lib/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs-tests.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-io-2.4.jar":"System Classpath","/hadoop/hadoop-nfs.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-yarn-api-2.7.4.jar":"System Classpath","/hadoop/lib/json-smart-1.3.1.jar":"System Classpath","/hadoop-yarn-client/lib/commons-lang-2.6.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/azure-keyvault-core-0.8.0.jar":"System Classpath","/hadoop-yarn-client/lib/fst-2.24.jar":"System Classpath","/hadoop-yarn-client/lib/htrace-core-3.1.0-incubating.jar":"System Classpath","/hadoop/hadoop-azure-datalake.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-common-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hive-exec-1.2.1.spark2.jar":"System Classpath","/hadoop-hdfs-client/lib/jersey-server-1.9.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-sharedcachemanager.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-launcher_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/datanucleus-core-3.2.10.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/zookeeper-3.4.6.jar":"System Classpath","/hadoop-yarn-client/lib/commons-math3-3.1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/api-util-1.0.0-M20.jar":"System Classpath","/hadoop/lib/azure-keyvault-core-0.8.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-common-2.22.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jtransforms-2.4.0.jar":"System Classpath","/hadoop/lib/asm-3.2.jar":"System Classpath","/hadoop/lib/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/javax.ws.rs-api-2.0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/xmlenc-0.52.jar":"System Classpath","/hadoop/lib/jsch-0.1.54.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-timeline-pluginstorage.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-annotations-2.2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/hdfs/lib/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/httpclient-4.5.6.jar":"System Classpath","/hadoop-hdfs-client/lib/xmlenc-0.52.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-annotations-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/antlr-runtime-3.4.jar":"System Classpath","/etc/hadoop/conf/secure":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/avro-ipc-1.8.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/janino-3.0.11.jar":"System Classpath","/hadoop/lib/api-util-1.0.0-M20.jar":"System Classpath","/hadoop-yarn-client/lib/xmlenc-0.52.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/slf4j-api-1.7.16.jar":"System Classpath","/hadoop/lib/ranger-hdfs-plugin-shim-0.7.0.jar":"System Classpath","/hadoop-hdfs-client/lib/guava-11.0.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/chill-java-0.9.3.jar":"System Classpath","/hadoop-yarn-client/lib/aopalliance-1.0.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-applications-unmanaged-am-launcher.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/htrace-core-3.1.0-incubating.jar":"System Classpath","/hadoop/lib/jersey-json-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/guava-14.0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-yarn-server-web-proxy-2.7.4.jar":"System Classpath","/hadoop/lib/jetty-sslengine-6.1.26.hwx.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/ST4-4.0.4.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-client-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-sketch_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-yarn-client/lib/httpclient-4.5.2.jar":"System Classpath","/hadoop/lib/httpclient-4.5.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/antlr4-runtime-4.7.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/arrow-vector-0.10.0.jar":"System Classpath","/hadoop-yarn-client/lib/jersey-server-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-yarn-client-2.7.4.jar":"System Classpath","/hadoop-yarn-client/lib/jsch-0.1.54.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/curator-client-2.7.1.jar":"System Classpath","/hadoop-yarn-client/lib/stax-api-1.0-2.jar":"System Classpath","/hadoop-yarn-client/lib/commons-cli-1.2.jar":"System Classpath","/hadoop/lib/servlet-api-2.5.jar":"System Classpath","/hadoop-yarn-client/lib/commons-collections-3.2.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-net-3.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-media-jaxb-2.22.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-mapreduce-client-app-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-core_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-catalyst_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-kvstore_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/lib/joda-time-2.9.4.jar":"System Classpath","/hadoop-yarn-client/lib/jersey-client-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-lang3-3.8.1.jar":"System Classpath","/hadoop/lib/slf4j-log4j12-1.7.10.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/bonecp-0.8.0.RELEASE.jar":"System Classpath","/hadoop-yarn-client/lib/jetty-sslengine-6.1.26.hwx.jar":"System Classpath","/hadoop-yarn-client/lib/commons-logging-1.1.3.jar":"System Classpath","/hadoop/lib/jets3t-0.9.0.jar":"System Classpath","/hadoop/lib/hadoop-lzo-0.6.0-javadoc.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/orc-shims-1.5.4.jar":"System Classpath","/hadoop-hdfs-client/lib/okio-1.6.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-column-1.10.0.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-core-asl-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/jsp-api-2.1.jar":"System Classpath","/hadoop/lib/aws-java-sdk-s3-1.10.6.jar":"System Classpath","/hadoop/azure-data-lake-store-sdk-2.2.5.jar":"System Classpath","/hadoop-yarn-client/lib/zookeeper-3.4.6-tests.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/stream-2.7.0.jar":"System Classpath","/hadoop/hadoop-azure-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/jcip-annotations-1.0-1.jar":"System Classpath","/hadoop-yarn-client/lib/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/common/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/aircompressor-0.10.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/httpcore-4.4.10.jar":"System Classpath","/hadoop-yarn-client/lib/objenesis-2.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/avro-1.8.2.jar":"System Classpath","/hadoop-yarn-client/lib/jets3t-0.9.0.jar":"System Classpath","/hadoop-hdfs-client/lib/jackson-core-2.2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-configuration-1.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-mapreduce-client-jobclient-2.7.4.jar":"System Classpath","/hadoop-yarn-client/lib/commons-io-2.4.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-tests.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/guice-3.0.jar":"System Classpath","/hadoop/lib/jsr305-3.0.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/javassist-3.18.1-GA.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/curator-framework-2.7.1.jar":"System Classpath","/hadoop-yarn-client/lib/avro-1.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/scala-compiler-2.12.8.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-registry.jar":"System Classpath","/hadoop/lib/stax-api-1.0-2.jar":"System Classpath","/hadoop-yarn-client/lib/guice-servlet-3.0.jar":"System Classpath","/hadoop/lib/commons-math3-3.1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/javax.annotation-api-1.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jdo-api-3.0.1.jar":"System Classpath","/hadoop-hdfs-client/lib/jackson-core-asl-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-sql_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-yarn-server-common-2.7.4.jar":"System Classpath","/hadoop-yarn-client/lib/api-util-1.0.0-M20.jar":"System Classpath","/hadoop/lib/api-asn1-api-1.0.0-M20.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/minlog-1.3.0.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-xc-1.9.13.jar":"System Classpath","/hadoop-hdfs-client/lib/netty-3.6.2.Final.jar":"System Classpath","/hadoop/lib/hamcrest-core-1.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/xbean-asm7-shaded-4.12.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-core-2.2.3.jar":"System Classpath","/hadoop/lib/jaxb-impl-2.2.3-1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/scala-parser-combinators_2.12-1.1.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-io-2.4.jar":"System Classpath","/hadoop/lib/commons-lang-2.6.jar":"System Classpath","/hadoop/lib/commons-lang3-3.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spire_2.12-0.13.0.jar":"System Classpath","/hadoop-hdfs-client/lib/jackson-annotations-2.2.3.jar":"System Classpath","/hadoop/lib/jackson-core-asl-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/javax.servlet-api-3.1.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/core-1.1.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-cli-1.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/curator-recipes-2.7.1.jar":"System Classpath","/hadoop/lib/jersey-server-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spire-macros_2.12-0.13.0.jar":"System Classpath","/hadoop-yarn-client/lib/javassist-3.18.1-GA.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/validation-api-1.1.0.Final.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/protobuf-java-2.5.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-beanutils-core-1.8.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/gson-2.2.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/ivy-2.4.0.jar":"System Classpath","/hadoop-yarn-client/lib/paranamer-2.3.jar":"System Classpath","/hadoop/lib/gson-2.2.4.jar":"System Classpath","/hadoop/lib/jackson-annotations-2.2.3.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-databind-2.2.3.jar":"System Classpath","/hadoop-hdfs-client/lib/leveldbjni-all-1.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/guice-servlet-3.0.jar":"System Classpath","/hadoop/hadoop-annotations.jar":"System Classpath","/hadoop-yarn-client/lib/curator-recipes-2.7.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/flatbuffers-1.2.0-3f79e055.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-mllib_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-tests-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-common-1.10.0.jar":"System Classpath","/hadoop-yarn-client/lib/asm-3.2.jar":"System Classpath","/hadoop/lib/protobuf-java-2.5.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-core-2.9.6.jar":"System Classpath","/hadoop/lib/commons-configuration-1.6.jar":"System Classpath"}} -{"Event":"SparkListenerApplicationStart","App Name":"Spark SQL basic example","App ID":"application_1547723113049_0005","Timestamp":1547788444248,"User":"spark","App Attempt ID":"1","Driver Logs":{"stdout":"http://node2:8042/node/containerlogs/container_e01_1547723113049_0005_01_000001/spark/stdout?start=-4096","stderr":"http://node2:8042/node/containerlogs/container_e01_1547723113049_0005_01_000001/spark/stderr?start=-4096"}} +{"Event":"SparkListenerApplicationStart","App Name":"Spark SQL basic example","App ID":"application_1547723113049_0005","Timestamp":1547788444248,"User":"spark","App Attempt ID":"1","Driver Logs":{"stdout":"http://node2:8042/node/containerlogs/container_e01_1547723113049_0005_01_000001/spark/stdout?start=-4096","stderr":"http://node2:8042/node/containerlogs/container_e01_1547723113049_0005_01_000001/spark/stderr?start=-4096"},"Driver Attributes":{"NODE_HTTP_ADDRESS":"node2:8042","USER":"spark","LOG_FILES":"stderr,stdout","CLUSTER_ID":"yarn-cluster","HTTP_SCHEME":"http://","CONTAINER_ID":"container_e01_1547723113049_0005_01_000001"}} {"Event":"SparkListenerExecutorAdded","Timestamp":1547788454209,"Executor ID":"1","Executor Info":{"Host":"node3","Total Cores":1,"Log Urls":{"stdout":"http://node3:8042/node/containerlogs/container_e01_1547723113049_0005_01_000002/spark/stdout?start=-4096","stderr":"http://node3:8042/node/containerlogs/container_e01_1547723113049_0005_01_000002/spark/stderr?start=-4096"},"Attributes":{"NODE_HTTP_ADDRESS":"node3:8042","USER":"spark","LOG_FILES":"stderr,stdout","CLUSTER_ID":"yarn-cluster","HTTP_SCHEME":"http://","CONTAINER_ID":"container_e01_1547723113049_0005_01_000002"}}} {"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"node3","Port":35087},"Maximum Memory":384093388,"Timestamp":1547788454413,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} {"Event":"SparkListenerExecutorAdded","Timestamp":1547788455343,"Executor ID":"2","Executor Info":{"Host":"node4","Total Cores":1,"Log Urls":{"stdout":"http://node4:8042/node/containerlogs/container_e01_1547723113049_0005_01_000003/spark/stdout?start=-4096","stderr":"http://node4:8042/node/containerlogs/container_e01_1547723113049_0005_01_000003/spark/stderr?start=-4096"},"Attributes":{"NODE_HTTP_ADDRESS":"node4:8042","USER":"spark","LOG_FILES":"stderr,stdout","CLUSTER_ID":"yarn-cluster","HTTP_SCHEME":"http://","CONTAINER_ID":"container_e01_1547723113049_0005_01_000003"}}} diff --git a/core/src/test/resources/spark-events/application_1547723113049_0006 b/core/src/test/resources/spark-events/application_1547723113049_0006 index 10b9c6eb10f7f..12124f4f4a710 100644 --- a/core/src/test/resources/spark-events/application_1547723113049_0006 +++ b/core/src/test/resources/spark-events/application_1547723113049_0006 @@ -1,7 +1,7 @@ {"Event":"SparkListenerLogStart","Spark Version":"3.0.0-SNAPSHOT"} {"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"node2","Port":40675},"Maximum Memory":428762726,"Timestamp":1547788445481,"Maximum Onheap Memory":428762726,"Maximum Offheap Memory":0} {"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre","Java Version":"1.8.0_161 (Oracle Corporation)","Scala Version":"version 2.12.8"},"Spark Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0006","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.scheduler.mode":"FIFO","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0006","spark.app.id":"application_1547723113049_0006","spark.hadoop.yarn.timeline-service.enabled":"false"},"System Properties":{"java.io.tmpdir":"/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_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/yarn","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/amd64","user.dir":"/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001","java.library.path":"/usr/java/packages/lib/amd64:/usr/lib64:/lib64:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.161-b14","jetty.git.hash":"27208684755d94a92186989f695db2d7b21ebc51","java.endorsed.dirs":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/endorsed","java.runtime.version":"1.8.0_161-b14","java.vm.info":"mixed mode","java.ext.dirs":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"OpenJDK Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/resources.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/rt.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/sunrsasign.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/jsse.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/jce.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/charsets.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/jfr.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/classes","file.encoding":"UTF-8","user.timezone":"UTC","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"4.18.5-1.el7.elrepo.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":"yarn","hdp.version":"2.6.5.1002-73","java.vm.name":"OpenJDK 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.yarn.ApplicationMaster --class org.apache.spark.examples.sql.SparkSQLExample --jar file:/home/spark/spark-3.0.0-snapshot-spark-26311-v2/examples/jars/spark-examples_2.12-3.0.0-SNAPSHOT.jar --properties-file /hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_conf__/__spark_conf__.properties --dist-cache-conf /hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_conf__/__spark_dist_cache__.properties","java.home":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre","java.version":"1.8.0_161","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/hadoop-yarn-client/lib/commons-digester-1.8.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-applications-distributedshell-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-compiler-3.0.11.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-hadoop-bundle-1.6.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/lz4-java-1.5.0.jar":"System Classpath","/hadoop/lib/commons-digester-1.8.jar":"System Classpath","/hadoop-yarn-client/lib/netty-3.6.2.Final.jar":"System Classpath","/hadoop/lib/curator-client-2.7.1.jar":"System Classpath","/hadoop/lib/log4j-1.2.17.jar":"System Classpath","/hadoop/lib/paranamer-2.3.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-codec-1.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hk2-api-2.4.0-b34.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/orc-core-1.5.4-nohive.jar":"System Classpath","/hadoop-yarn-client/lib/curator-client-2.7.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop/hadoop-common.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-cli-1.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-hdfs-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/aopalliance-1.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-guava-2.22.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/stax-api-1.0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/snappy-java-1.1.7.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/univocity-parsers-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/azure-storage-5.4.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/netty-all-4.1.30.Final.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-container-servlet-core-2.22.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/macro-compat_2.12-1.1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-httpclient-3.1.jar":"System Classpath","/hadoop/lib/httpcore-4.4.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/hadoop/lib/java-xmlbuilder-0.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jodd-core-3.5.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/breeze-macros_2.12-0.13.2.jar":"System Classpath","/hadoop-yarn-client/lib/zookeeper-3.4.6.jar":"System Classpath","/hadoop/lib/curator-framework-2.7.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jul-to-slf4j-1.7.16.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/xz-1.5.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/antlr-2.7.7.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/json4s-scalap_2.12-3.5.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/apache-log4j-extras-1.2.17.jar":"System Classpath","/hadoop-hdfs-client/lib/htrace-core-3.1.0-incubating.jar":"System Classpath","/hadoop/hadoop-azure.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-nodemanager.jar":"System Classpath","/hadoop-yarn-client/lib/servlet-api-2.5.jar":"System Classpath","/hadoop-yarn-client/lib/guava-11.0.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/arpack_combined_all-0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/arrow-format-0.10.0.jar":"System Classpath","/hadoop-yarn-client/lib/guice-3.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/scala-library-2.12.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-container-servlet-2.22.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/xercesImpl-2.9.1.jar":"System Classpath","/hadoop/hadoop-azure-datalake-2.7.3.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-common.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/yarn/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-auth-2.7.4.jar":"System Classpath","/hadoop/lib/activation-1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jline-2.14.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/yarn/lib/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hive-metastore-1.2.1.spark2.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-applicationhistoryservice.jar":"System Classpath","/hadoop/lib/ojdbc6.jar":"System Classpath","/hadoop/lib/commons-collections-3.2.2.jar":"System Classpath","/hadoop-hdfs-client/lib/servlet-api-2.5.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/joda-time-2.9.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-annotations-2.9.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-mllib-local_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/leveldbjni-all-1.8.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-common-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/json4s-jackson_2.12-3.5.3.jar":"System Classpath","/hadoop/conf":"System Classpath","/hadoop/hadoop-common-2.7.3-tests.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-timeline-pluginstorage-2.7.3.jar":"System Classpath","/hadoop/lib/junit-4.11.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs-2.7.3-tests.jar":"System Classpath","/hadoop-yarn-client/lib/httpcore-4.4.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-unsafe_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/metrics-jvm-3.1.5.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/objenesis-2.5.1.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-nodemanager-2.7.3.jar":"System Classpath","/hadoop/hadoop-common-tests.jar":"System Classpath","/hadoop-hdfs-client/lib/jackson-databind-2.2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-graphx_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-hdfs-client/lib/asm-3.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/kryo-shaded-4.0.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/compress-lzf-1.0.3.jar":"System Classpath","/hadoop/lib/azure-storage-5.4.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/avro-mapred-1.8.2-hadoop2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/scala-xml_2.12-1.0.5.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-applications-distributedshell.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jaxb-api-2.2.2.jar":"System Classpath","/hadoop/lib/commons-net-3.1.jar":"System Classpath","/hadoop-yarn-client/lib/java-xmlbuilder-0.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/paranamer-2.8.jar":"System Classpath","/hadoop/lib/commons-logging-1.1.3.jar":"System Classpath","/hadoop-yarn-client/lib/log4j-1.2.17.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/mapreduce/*":"System Classpath","/hadoop/lib/commons-codec-1.4.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs-nfs-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/commons-lang3-3.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-math3-3.4.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-network-common_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jetty-6.1.26.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/javolution-5.5.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jcl-over-slf4j-1.7.16.jar":"System Classpath","/hadoop/lib/slf4j-api-1.7.10.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-common-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-core-asl-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hk2-utils-2.4.0-b34.jar":"System Classpath","/hadoop/lib/mockito-all-1.8.5.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-client.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/common/lib/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hppc-0.7.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/shapeless_2.12-2.3.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-mapreduce-client-common-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/log4j-1.2.17.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-module-scala_2.12-2.9.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-module-paranamer-2.9.6.jar":"System Classpath","/hadoop-hdfs-client/lib/jetty-6.1.26.hwx.jar":"System Classpath","/hadoop/hadoop-aws.jar":"System Classpath","/hadoop/lib/commons-cli-1.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-codec-1.10.jar":"System Classpath","/hadoop/lib/commons-compress-1.4.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-server-2.22.2.jar":"System Classpath","/hadoop/lib/avro-1.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-xc-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-dbcp-1.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-beanutils-1.7.0.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-jaxrs-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/api-asn1-api-1.0.0-M20.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-client-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/javax.inject-2.4.0-b34.jar":"System Classpath","/hadoop/gcs-connector-1.8.1-shaded.jar":"System Classpath","/hadoop-hdfs-client/lib/jersey-core-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/metrics-json-3.1.5.jar":"System Classpath","/hadoop/lib/jackson-core-2.2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jpam-1.1.jar":"System Classpath","/hadoop/lib/jersey-core-1.9.jar":"System Classpath","/hadoop/lib/snappy-java-1.0.4.1.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-lang-2.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/slf4j-log4j12-1.7.16.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/osgi-resource-locator-1.0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/api-asn1-api-1.0.0-M20.jar":"System Classpath","/hadoop-yarn-client/lib/nimbus-jose-jwt-4.41.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-databind-2.9.6.jar":"System Classpath","/hadoop-hdfs-client/lib/xml-apis-1.3.04.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-client-2.22.2.jar":"System Classpath","/hadoop/hadoop-nfs-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hive-jdbc-1.2.1.spark2.jar":"System Classpath","/hadoop/lib/jackson-databind-2.2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/eigenbase-properties-1.1.5.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/machinist_2.12-0.6.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/tools/lib/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-jackson-1.10.0.jar":"System Classpath","/hadoop-yarn-client/lib/jetty-util-6.1.26.hwx.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-digester-1.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/calcite-avatica-1.2.0-incubating.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/oro-2.0.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hive-cli-1.2.1.spark2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-hive-thriftserver_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-applications-unmanaged-am-launcher-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/JavaEWAH-0.3.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/super-csv-2.2.0.jar":"System Classpath","/hadoop/lib/ranger-yarn-plugin-shim-0.7.0.jar":"System Classpath","/hadoop-yarn-client/lib/jetty-6.1.26.hwx.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/metrics-core-3.1.5.jar":"System Classpath","/hadoop-yarn-client/lib/commons-compress-1.4.1.jar":"System Classpath","/hadoop-hdfs-client/lib/netty-all-4.0.52.Final.jar":"System Classpath","/hadoop/hadoop-auth.jar":"System Classpath","/hadoop-yarn-client/lib/javax.inject-1.jar":"System Classpath","/hadoop/lib/jettison-1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jetty-util-6.1.26.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-network-shuffle_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/lib/jackson-jaxrs-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/json-smart-1.3.1.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-api-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/commons-beanutils-1.7.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-streaming_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/lib/commons-io-2.4.jar":"System Classpath","/hadoop/lib/commons-beanutils-1.7.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/snappy-0.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jta-1.1.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-resourcemanager-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/protobuf-java-2.5.0.jar":"System Classpath","/hadoop-yarn-client/lib/curator-framework-2.7.1.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-daemon-1.0.13.jar":"System Classpath","/hadoop/lib/hadoop-lzo-0.6.0-sources.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-common.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/breeze_2.12-0.13.2.jar":"System Classpath","/hadoop/lib/jsp-api-2.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/orc-mapreduce-1.5.4-nohive.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-mapreduce-client-core-2.7.4.jar":"System Classpath","/hadoop/lib/htrace-core-3.1.0-incubating.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-encoding-1.10.0.jar":"System Classpath","/hadoop/hadoop-auth-2.7.3.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-applicationhistoryservice-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/jettison-1.1.jar":"System Classpath","/hadoop-yarn-client/lib/commons-codec-1.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-mapreduce-client-shuffle-2.7.4.jar":"System Classpath","/hadoop/lib/curator-recipes-2.7.1.jar":"System Classpath","/hadoop/lib/jetty-util-6.1.26.hwx.jar":"System Classpath","/hadoop-hdfs-client/lib/log4j-1.2.17.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/activation-1.1.1.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/jaxb-api-2.2.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/opencsv-2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-collections-3.2.2.jar":"System Classpath","/hadoop/lib/guava-11.0.2.jar":"System Classpath","/hadoop-yarn-client/lib/commons-net-3.1.jar":"System Classpath","/hadoop/lib/commons-beanutils-core-1.8.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/py4j-0.10.8.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/json4s-ast_2.12-3.5.3.jar":"System Classpath","/hadoop-yarn-client/lib/metrics-core-3.0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/pyrolite-4.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/chill_2.12-0.9.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/libfb303-0.9.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/javax.inject-1.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs.jar":"System Classpath","/ext/hadoop/*":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-web-proxy-2.7.3.jar":"System Classpath","/hadoop/lib/aws-java-sdk-core-1.10.6.jar":"System Classpath","/hadoop-yarn-client/lib/commons-configuration-1.6.jar":"System Classpath","/hadoop/lib/zookeeper-3.4.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-logging-1.1.3.jar":"System Classpath","/hadoop-hdfs-client/lib/jsr305-3.0.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/stringtemplate-3.2.1.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-resourcemanager.jar":"System Classpath","/hadoop/lib/netty-3.6.2.Final.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/zstd-jni-1.3.2-2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jsp-api-2.1.jar":"System Classpath","/hadoop-yarn-client/lib/jaxb-impl-2.2.3-1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-tags_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hive-beeline-1.2.1.spark2.jar":"System Classpath","/hadoop-hdfs-client/lib/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/snappy-java-1.0.4.1.jar":"System Classpath","/hadoop-yarn-client/lib/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs-nfs.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/hadoop/hadoop-aws-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/jsr305-3.0.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/datanucleus-rdbms-3.2.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/libthrift-0.9.3.jar":"System Classpath","/hadoop/lib/xz-1.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/json4s-core_2.12-3.5.3.jar":"System Classpath","/hadoop-yarn-client/lib/jersey-guice-1.9.jar":"System Classpath","/hadoop-yarn-client/lib/activation-1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-jaxrs-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/hdfs/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/mapreduce/lib/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-pool-1.5.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/arrow-memory-0.10.0.jar":"System Classpath","/hadoop-hdfs-client/lib/jetty-util-6.1.26.hwx.jar":"System Classpath","/hadoop/lib/ranger-plugin-classloader-0.7.0.jar":"System Classpath","/hadoop/lib/aws-java-sdk-kms-1.10.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-repl_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-yarn-client/lib/jersey-json-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-yarn_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-yarn-client/lib/xz-1.0.jar":"System Classpath","/hadoop/hadoop-common-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jetty-sslengine-6.1.26.jar":"System Classpath","/hadoop/lib/jaxb-api-2.2.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/metrics-graphite-3.1.5.jar":"System Classpath","/hadoop/lib/jcip-annotations-1.0-1.jar":"System Classpath","/hadoop/lib/nimbus-jose-jwt-4.41.1.jar":"System Classpath","/hadoop/lib/hadoop-lzo-0.6.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hk2-locator-2.4.0-b34.jar":"System Classpath","/hadoop/lib/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-logging-1.1.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jsr305-3.0.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-hive_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/RoaringBitmap-0.5.11.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/calcite-linq4j-1.2.0-incubating.jar":"System Classpath","/hadoop-yarn-client/lib/commons-beanutils-core-1.8.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-lang-2.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/derby-10.12.1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-hadoop-1.10.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/stax-api-1.0-2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/scala-reflect-2.12.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/calcite-core-1.2.0-incubating.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_conf__":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-format-2.4.0.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-sharedcachemanager-2.7.3.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-web-proxy.jar":"System Classpath","/hadoop-hdfs-client/lib/xercesImpl-2.9.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/netty-3.9.9.Final.jar":"System Classpath","/hadoop/lib/xmlenc-0.52.jar":"System Classpath","/hadoop/hadoop-annotations-2.7.3.jar":"System Classpath","/hadoop-hdfs-client/lib/protobuf-java-2.5.0.jar":"System Classpath","/hadoop-yarn-client/lib/jersey-core-1.9.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-registry-2.7.3.jar":"System Classpath","/hadoop-hdfs-client/lib/okhttp-2.7.5.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-yarn-common-2.7.4.jar":"System Classpath","/hadoop-yarn-client/lib/leveldbjni-all-1.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-crypto-1.0.0.jar":"System Classpath","/hadoop/lib/jetty-6.1.26.hwx.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/aopalliance-repackaged-2.4.0-b34.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_conf__/__hadoop_conf__":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-compress-1.8.1.jar":"System Classpath","/hadoop/lib/jackson-xc-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/gson-2.2.4.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-api.jar":"System Classpath","/hadoop/lib/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs-tests.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-io-2.4.jar":"System Classpath","/hadoop/hadoop-nfs.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-yarn-api-2.7.4.jar":"System Classpath","/hadoop/lib/json-smart-1.3.1.jar":"System Classpath","/hadoop-yarn-client/lib/commons-lang-2.6.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/azure-keyvault-core-0.8.0.jar":"System Classpath","/hadoop-yarn-client/lib/fst-2.24.jar":"System Classpath","/hadoop-yarn-client/lib/htrace-core-3.1.0-incubating.jar":"System Classpath","/hadoop/hadoop-azure-datalake.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-common-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hive-exec-1.2.1.spark2.jar":"System Classpath","/hadoop-hdfs-client/lib/jersey-server-1.9.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-sharedcachemanager.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-launcher_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/datanucleus-core-3.2.10.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/zookeeper-3.4.6.jar":"System Classpath","/hadoop-yarn-client/lib/commons-math3-3.1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/api-util-1.0.0-M20.jar":"System Classpath","/hadoop/lib/azure-keyvault-core-0.8.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-common-2.22.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jtransforms-2.4.0.jar":"System Classpath","/hadoop/lib/asm-3.2.jar":"System Classpath","/hadoop/lib/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/javax.ws.rs-api-2.0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/xmlenc-0.52.jar":"System Classpath","/hadoop/lib/jsch-0.1.54.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-timeline-pluginstorage.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-annotations-2.2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/hdfs/lib/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/httpclient-4.5.6.jar":"System Classpath","/hadoop-hdfs-client/lib/xmlenc-0.52.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-annotations-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/antlr-runtime-3.4.jar":"System Classpath","/etc/hadoop/conf/secure":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/avro-ipc-1.8.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/janino-3.0.11.jar":"System Classpath","/hadoop/lib/api-util-1.0.0-M20.jar":"System Classpath","/hadoop-yarn-client/lib/xmlenc-0.52.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/slf4j-api-1.7.16.jar":"System Classpath","/hadoop/lib/ranger-hdfs-plugin-shim-0.7.0.jar":"System Classpath","/hadoop-hdfs-client/lib/guava-11.0.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/chill-java-0.9.3.jar":"System Classpath","/hadoop-yarn-client/lib/aopalliance-1.0.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-applications-unmanaged-am-launcher.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/htrace-core-3.1.0-incubating.jar":"System Classpath","/hadoop/lib/jersey-json-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/guava-14.0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-yarn-server-web-proxy-2.7.4.jar":"System Classpath","/hadoop/lib/jetty-sslengine-6.1.26.hwx.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/ST4-4.0.4.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-client-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-sketch_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-yarn-client/lib/httpclient-4.5.2.jar":"System Classpath","/hadoop/lib/httpclient-4.5.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/antlr4-runtime-4.7.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/arrow-vector-0.10.0.jar":"System Classpath","/hadoop-yarn-client/lib/jersey-server-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-yarn-client-2.7.4.jar":"System Classpath","/hadoop-yarn-client/lib/jsch-0.1.54.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/curator-client-2.7.1.jar":"System Classpath","/hadoop-yarn-client/lib/stax-api-1.0-2.jar":"System Classpath","/hadoop-yarn-client/lib/commons-cli-1.2.jar":"System Classpath","/hadoop/lib/servlet-api-2.5.jar":"System Classpath","/hadoop-yarn-client/lib/commons-collections-3.2.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-net-3.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-media-jaxb-2.22.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-mapreduce-client-app-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-core_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-catalyst_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-kvstore_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/lib/joda-time-2.9.4.jar":"System Classpath","/hadoop-yarn-client/lib/jersey-client-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-lang3-3.8.1.jar":"System Classpath","/hadoop/lib/slf4j-log4j12-1.7.10.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/bonecp-0.8.0.RELEASE.jar":"System Classpath","/hadoop-yarn-client/lib/jetty-sslengine-6.1.26.hwx.jar":"System Classpath","/hadoop-yarn-client/lib/commons-logging-1.1.3.jar":"System Classpath","/hadoop/lib/jets3t-0.9.0.jar":"System Classpath","/hadoop/lib/hadoop-lzo-0.6.0-javadoc.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/orc-shims-1.5.4.jar":"System Classpath","/hadoop-hdfs-client/lib/okio-1.6.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-column-1.10.0.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-core-asl-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/jsp-api-2.1.jar":"System Classpath","/hadoop/lib/aws-java-sdk-s3-1.10.6.jar":"System Classpath","/hadoop/azure-data-lake-store-sdk-2.2.5.jar":"System Classpath","/hadoop-yarn-client/lib/zookeeper-3.4.6-tests.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/stream-2.7.0.jar":"System Classpath","/hadoop/hadoop-azure-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/jcip-annotations-1.0-1.jar":"System Classpath","/hadoop-yarn-client/lib/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/common/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/aircompressor-0.10.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/httpcore-4.4.10.jar":"System Classpath","/hadoop-yarn-client/lib/objenesis-2.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/avro-1.8.2.jar":"System Classpath","/hadoop-yarn-client/lib/jets3t-0.9.0.jar":"System Classpath","/hadoop-hdfs-client/lib/jackson-core-2.2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-configuration-1.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-mapreduce-client-jobclient-2.7.4.jar":"System Classpath","/hadoop-yarn-client/lib/commons-io-2.4.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-tests.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/guice-3.0.jar":"System Classpath","/hadoop/lib/jsr305-3.0.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/javassist-3.18.1-GA.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/curator-framework-2.7.1.jar":"System Classpath","/hadoop-yarn-client/lib/avro-1.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/scala-compiler-2.12.8.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-registry.jar":"System Classpath","/hadoop/lib/stax-api-1.0-2.jar":"System Classpath","/hadoop-yarn-client/lib/guice-servlet-3.0.jar":"System Classpath","/hadoop/lib/commons-math3-3.1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/javax.annotation-api-1.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jdo-api-3.0.1.jar":"System Classpath","/hadoop-hdfs-client/lib/jackson-core-asl-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-sql_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-yarn-server-common-2.7.4.jar":"System Classpath","/hadoop-yarn-client/lib/api-util-1.0.0-M20.jar":"System Classpath","/hadoop/lib/api-asn1-api-1.0.0-M20.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/minlog-1.3.0.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-xc-1.9.13.jar":"System Classpath","/hadoop-hdfs-client/lib/netty-3.6.2.Final.jar":"System Classpath","/hadoop/lib/hamcrest-core-1.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/xbean-asm7-shaded-4.12.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-core-2.2.3.jar":"System Classpath","/hadoop/lib/jaxb-impl-2.2.3-1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/scala-parser-combinators_2.12-1.1.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-io-2.4.jar":"System Classpath","/hadoop/lib/commons-lang-2.6.jar":"System Classpath","/hadoop/lib/commons-lang3-3.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spire_2.12-0.13.0.jar":"System Classpath","/hadoop-hdfs-client/lib/jackson-annotations-2.2.3.jar":"System Classpath","/hadoop/lib/jackson-core-asl-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/javax.servlet-api-3.1.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/core-1.1.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-cli-1.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/curator-recipes-2.7.1.jar":"System Classpath","/hadoop/lib/jersey-server-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spire-macros_2.12-0.13.0.jar":"System Classpath","/hadoop-yarn-client/lib/javassist-3.18.1-GA.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/validation-api-1.1.0.Final.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/protobuf-java-2.5.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-beanutils-core-1.8.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/gson-2.2.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/ivy-2.4.0.jar":"System Classpath","/hadoop-yarn-client/lib/paranamer-2.3.jar":"System Classpath","/hadoop/lib/gson-2.2.4.jar":"System Classpath","/hadoop/lib/jackson-annotations-2.2.3.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-databind-2.2.3.jar":"System Classpath","/hadoop-hdfs-client/lib/leveldbjni-all-1.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/guice-servlet-3.0.jar":"System Classpath","/hadoop/hadoop-annotations.jar":"System Classpath","/hadoop-yarn-client/lib/curator-recipes-2.7.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/flatbuffers-1.2.0-3f79e055.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-mllib_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-tests-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-common-1.10.0.jar":"System Classpath","/hadoop-yarn-client/lib/asm-3.2.jar":"System Classpath","/hadoop/lib/protobuf-java-2.5.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-core-2.9.6.jar":"System Classpath","/hadoop/lib/commons-configuration-1.6.jar":"System Classpath"}} -{"Event":"SparkListenerApplicationStart","App Name":"Spark SQL basic example","App ID":"application_1547723113049_0006","Timestamp":1547788454200,"User":"spark","App Attempt ID":"1","Driver Logs":{"stdout":"http://node2:8042/node/containerlogs/container_e01_1547723113049_0005_01_000001/spark/stdout?start=-4096","stderr":"http://node2:8042/node/containerlogs/container_e01_1547723113049_0005_01_000001/spark/stderr?start=-4096"}} +{"Event":"SparkListenerApplicationStart","App Name":"Spark SQL basic example","App ID":"application_1547723113049_0006","Timestamp":1547788454200,"User":"spark","App Attempt ID":"1","Driver Logs":{"stdout":"http://node2:8042/node/containerlogs/container_e01_1547723113049_0005_01_000001/spark/stdout?start=-4096","stderr":"http://node2:8042/node/containerlogs/container_e01_1547723113049_0005_01_000001/spark/stderr?start=-4096"},"Driver Attributes":{"NODE_HTTP_ADDRESS":"node2:8042","USER":"spark","LOG_FILES":"stderr,stdout","CLUSTER_ID":"yarn-cluster","HTTP_SCHEME":"http://","CONTAINER_ID":"container_e01_1547723113049_0005_01_000001"}} {"Event":"SparkListenerExecutorAdded","Timestamp":1547788454209,"Executor ID":"1","Executor Info":{"Host":"node3","Total Cores":1,"Log Urls":{"stdout":"http://node3:8042/node/containerlogs/container_e01_1547723113049_0005_01_000002/spark/stdout?start=-4096","stderr":"http://node3:8042/node/containerlogs/container_e01_1547723113049_0005_01_000002/spark/stderr?start=-4096"},"Attributes":{"NODE_HTTP_ADDRESS":"node3:8042","USER":"spark","LOG_FILES":"stderr,stdout","CLUSTER_ID":"yarn-cluster","HTTP_SCHEME":"http://","CONTAINER_ID":"container_e01_1547723113049_0005_01_000002"}}} {"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"node3","Port":35087},"Maximum Memory":384093388,"Timestamp":1547788454413,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} {"Event":"SparkListenerExecutorAdded","Timestamp":1547788455343,"Executor ID":"2","Executor Info":{"Host":"node4","Total Cores":1,"Log Urls":{"stdout":"http://node4:8042/node/containerlogs/container_e01_1547723113049_0005_01_000003/spark/stdout?start=-4096","stderr":"http://node4:8042/node/containerlogs/container_e01_1547723113049_0005_01_000003/spark/stderr?start=-4096"},"Attributes":{"NODE_HTTP_ADDRESS":"node4:8042","USER":"spark","LOG_FILES":"stderr,stdout","CLUSTER_ID":"yarn-cluster","HTTP_SCHEME":"http://","CONTAINER_ID":"container_e01_1547723113049_0005_01_000003"}}} 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 897d3e462fcde..4ae784e5767ee 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 @@ -193,7 +193,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers "completed app executor list json apply custom log urls" -> "applications/application_1547723113049_0005/1/executors", "incomplete app executor list json apply custom log urls" -> - "applications/application_1547723113049_0006/1/executors", + "applications/application_1547723113049_0006/1/executors" ) casesCustomLogUrl.foreach { case (name, path) => @@ -209,9 +209,8 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers init( // compiler complains when using `->`, so made a tuple (CUSTOM_EXECUTOR_LOG_URL.key, "http://newhost:9999/logs/clusters/" + - "{{CLUSTER_ID}}/users/{{USER}}/containers/{{CONTAINER_ID}}/{{FILE_NAME}})"), - APPLY_CUSTOM_EXECUTOR_LOG_URL_TO_INCOMPLETE_APP.key -> applyToIncompleteApp.toString - ) + "{{CLUSTER_ID}}/users/{{USER}}/containers/{{CONTAINER_ID}}/{{FILE_NAME}}"), + APPLY_CUSTOM_EXECUTOR_LOG_URL_TO_INCOMPLETE_APP.key -> applyToIncompleteApp.toString) val expectationFile = new File(expRoot, HistoryServerSuite.sanitizePath(newName) + "_expectation.json") diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala index e6680e1829588..f86118e9994d0 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala @@ -17,8 +17,10 @@ package org.apache.spark.scheduler.cluster +import org.apache.hadoop.HadoopIllegalArgumentException import org.apache.hadoop.yarn.api.ApplicationConstants.Environment import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.util.ConverterUtils import org.apache.spark.SparkContext import org.apache.spark.deploy.SparkHadoopUtil @@ -66,4 +68,42 @@ private[spark] class YarnClusterSchedulerBackend( } driverLogs } + + override def getDriverAttributes: Option[Map[String, String]] = { + var attributes: Option[Map[String, String]] = None + try { + val yarnConf = new YarnConfiguration(sc.hadoopConfiguration) + val containerId = YarnSparkHadoopUtil.getContainerId + val clusterId: Option[String] = try { + Some(YarnConfiguration.getClusterId(yarnConf)) + } catch { + case _: HadoopIllegalArgumentException => None + } + + val httpAddress = System.getenv(Environment.NM_HOST.name()) + + ":" + System.getenv(Environment.NM_HTTP_PORT.name()) + + // lookup appropriate http scheme for container log urls + val yarnHttpPolicy = yarnConf.get( + YarnConfiguration.YARN_HTTP_POLICY_KEY, + YarnConfiguration.YARN_HTTP_POLICY_DEFAULT + ) + val user = Utils.getCurrentUserName() + val httpScheme = if (yarnHttpPolicy == "HTTPS_ONLY") "https://" else "http://" + + attributes = Some(Map( + "HTTP_SCHEME" -> httpScheme, + "NODE_HTTP_ADDRESS" -> httpAddress, + "CLUSTER_ID" -> clusterId.getOrElse(""), + "CONTAINER_ID" -> ConverterUtils.toString(containerId), + "USER" -> user, + "LOG_FILES" -> "stderr,stdout" + )) + } catch { + case e: Exception => + logInfo("Error while retrieving attributes on driver, so driver logs will not " + + "be replaced with custom log pattern", e) + } + attributes + } } diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index 70ad231e99262..cf2fc9bb7346e 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -28,6 +28,8 @@ import scala.io.Source import scala.language.postfixOps import com.google.common.io.{ByteStreams, Files} +import org.apache.hadoop.HadoopIllegalArgumentException +import org.apache.hadoop.yarn.api.ApplicationConstants import org.apache.hadoop.yarn.conf.YarnConfiguration import org.scalatest.Matchers import org.scalatest.concurrent.Eventually._ @@ -332,6 +334,7 @@ class YarnClusterSuite extends BaseYarnClusterSuite { private[spark] class SaveExecutorInfo extends SparkListener { val addedExecutorInfos = mutable.Map[String, ExecutorInfo]() var driverLogs: Option[collection.Map[String, String]] = None + var driverAttributes: Option[collection.Map[String, String]] = None override def onExecutorAdded(executor: SparkListenerExecutorAdded) { addedExecutorInfos(executor.executorId) = executor.executorInfo @@ -339,6 +342,7 @@ private[spark] class SaveExecutorInfo extends SparkListener { override def onApplicationStart(appStart: SparkListenerApplicationStart): Unit = { driverLogs = appStart.driverLogs + driverAttributes = appStart.driverAttributes } } @@ -443,6 +447,7 @@ private object YarnClusterDriver extends Logging with Matchers { s"Executor logs contain sensitive info (${SECRET_PASSWORD}): \n${log} " ) } + assert(info.attributes.nonEmpty) } // If we are running in yarn-cluster mode, verify that driver logs links and present and are @@ -461,9 +466,39 @@ private object YarnClusterDriver extends Logging with Matchers { s"Driver logs contain sensitive info (${SECRET_PASSWORD}): \n${log} " ) } + + val yarnConf = new YarnConfiguration(sc.hadoopConfiguration) + val httpAddress = System.getenv(ApplicationConstants.Environment.NM_HOST.name()) + + ":" + System.getenv(ApplicationConstants.Environment.NM_HTTP_PORT.name()) + + // lookup appropriate http scheme for container log urls + val yarnHttpPolicy = yarnConf.get( + YarnConfiguration.YARN_HTTP_POLICY_KEY, + YarnConfiguration.YARN_HTTP_POLICY_DEFAULT + ) + val httpScheme = if (yarnHttpPolicy == "HTTPS_ONLY") "https://" else "http://" + val containerId = YarnSparkHadoopUtil.getContainerId val user = Utils.getCurrentUserName() assert(urlStr.endsWith(s"/node/containerlogs/$containerId/$user/stderr?start=-4096")) + + assert(listener.driverAttributes.nonEmpty) + val driverAttributes = listener.driverAttributes.get + val clusterId: Option[String] = try { + Some(YarnConfiguration.getClusterId(yarnConf)) + } catch { + case _: HadoopIllegalArgumentException => None + } + + val expectationAttributes = Map( + "HTTP_SCHEME" -> httpScheme, + "NODE_HTTP_ADDRESS" -> httpAddress, + "CLUSTER_ID" -> clusterId, + "CONTAINER_ID" -> containerId, + "USER" -> user, + "LOG_FILES" -> "stderr,stdout") + + assert(driverAttributes === expectationAttributes) } } finally { Files.write(result, status, StandardCharsets.UTF_8) From 3d9cdb4ad83d725a5f785ed091b18cae9cfa9a70 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Sat, 19 Jan 2019 11:24:12 +0900 Subject: [PATCH 16/28] Add org.apache.spark.scheduler.SparkListenerApplicationStart to Mima exclude list --- project/MimaExcludes.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 0cdef0074cad3..3100d897375e4 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -1486,6 +1486,12 @@ object MimaExcludes { ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasLoss.org$apache$spark$ml$param$shared$HasLoss$_setter_$loss_="), ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasLoss.getLoss"), ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasLoss.loss") + ) ++ Seq( + // [SPARK-26311][CORE]New feature: apply custom log URL pattern for executor log URLs + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerApplicationStart.apply"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerApplicationStart.copy"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerApplicationStart.this"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.scheduler.SparkListenerApplicationStart$") ) } From 14eff2e01e4979f724cb8d0f053f7c67662cdfd4 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Sat, 19 Jan 2019 13:51:23 +0900 Subject: [PATCH 17/28] Fix UT in yarn --- .../org/apache/spark/deploy/yarn/YarnClusterSuite.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index cf2fc9bb7346e..0a6753715a859 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -31,6 +31,7 @@ import com.google.common.io.{ByteStreams, Files} import org.apache.hadoop.HadoopIllegalArgumentException import org.apache.hadoop.yarn.api.ApplicationConstants import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.util.ConverterUtils import org.scalatest.Matchers import org.scalatest.concurrent.Eventually._ @@ -493,8 +494,8 @@ private object YarnClusterDriver extends Logging with Matchers { val expectationAttributes = Map( "HTTP_SCHEME" -> httpScheme, "NODE_HTTP_ADDRESS" -> httpAddress, - "CLUSTER_ID" -> clusterId, - "CONTAINER_ID" -> containerId, + "CLUSTER_ID" -> clusterId.getOrElse(""), + "CONTAINER_ID" -> ConverterUtils.toString(containerId), "USER" -> user, "LOG_FILES" -> "stderr,stdout") From eb0ebaad301bcb4e008bc0a285d3080a0a0b02ae Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Wed, 23 Jan 2019 05:23:01 +0900 Subject: [PATCH 18/28] Address review comments from @squito --- .../org/apache/spark/internal/config/History.scala | 12 ++++++++---- .../spark/deploy/history/HistoryServerSuite.scala | 5 +++-- docs/monitoring.md | 13 ++++++++----- .../cluster/YarnClusterSchedulerBackend.scala | 5 ++--- 4 files changed, 21 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/History.scala b/core/src/main/scala/org/apache/spark/internal/config/History.scala index 7fa3a2954c13e..e57ebccff8e20 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/History.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/History.scala @@ -128,15 +128,19 @@ private[spark] object History { val CUSTOM_EXECUTOR_LOG_URL = ConfigBuilder("spark.history.custom.executor.log.url") .doc("Specifies custom spark executor log url for supporting external log service instead of " + - "using resource managers' app log urls. Spark will support some path variables via " + - "patterns which can be vary on resource managers. Please check the document for " + - "resource manager to refer which patterns are supported.") + "using cluster managers' application log urls in the history server. Spark will support " + + "some path variables via patterns which can vary on cluster manager. Please check the " + + "documentation for your each cluster manager to see which patterns are supported, if any. " + + "This configuration has no effect on a live application, it only affects the history server.") .stringConf .createOptional val APPLY_CUSTOM_EXECUTOR_LOG_URL_TO_INCOMPLETE_APP = ConfigBuilder("spark.history.custom.executor.log.url.applyIncompleteApplication") - .doc("Whether apply custom executor log url to incomplete application as well") + .doc("Whether apply custom executor log url, as specified by " + + "`spark.history.custom.executor.log.url`, to incomplete application as well. " + + "Even if this is true, this still only affects the behavior of the history server, " + + "not running spark applications.") .booleanConf .createWithDefault(true) } 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 4ae784e5767ee..fb358a1290b22 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 @@ -109,6 +109,8 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers } } + override protected def afterAll(): Unit = stop() + val cases = Seq( "application list json" -> "applications", "completed app list json" -> "applications?status=completed", @@ -207,8 +209,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers try { stop() init( - // compiler complains when using `->`, so made a tuple - (CUSTOM_EXECUTOR_LOG_URL.key, "http://newhost:9999/logs/clusters/" + + CUSTOM_EXECUTOR_LOG_URL.key -> ("http://newhost:9999/logs/clusters/" + "{{CLUSTER_ID}}/users/{{USER}}/containers/{{CONTAINER_ID}}/{{FILE_NAME}}"), APPLY_CUSTOM_EXECUTOR_LOG_URL_TO_INCOMPLETE_APP.key -> applyToIncompleteApp.toString) diff --git a/docs/monitoring.md b/docs/monitoring.md index 4617d9d6c5d0d..369fbc43f5e57 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -252,20 +252,23 @@ Security options for the Spark History Server are covered more detail in the spark.history.custom.executor.log.url (none) - Specifies custom spark executor log url for supporting external log service instead of using resource managers' app log urls. - Spark will support some path variables via patterns which can be vary on resource managers. - Please check the document for resource manager to refer which patterns are supported. + Specifies custom spark executor log url for supporting external log service instead of using cluster + managers' application log urls in the history server. Spark will support some path variables via patterns + which can vary on cluster manager. Please check the documentation for your each cluster manager to + see which patterns are supported, if any. This configuration has no effect on a live application, it only + affects the history server.

- For now, YARN mode only supports this configuration. + For now, only YARN mode supports this configuration spark.history.custom.executor.log.url.applyIncompleteApplication false - Specifies whether apply custom spark executor log url to incomplete applications as well. + Specifies whether to apply custom spark executor log url to incomplete applications as well. If executor logs for running applications should be provided as origin log urls, set this to `false`. Please note that incomplete applications may include applications which didn't shutdown gracefully. + Even this is set to `true`, this configuration has no effect on a live application, it only affects the history server. diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala index f86118e9994d0..42d266bfff91e 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala @@ -70,7 +70,6 @@ private[spark] class YarnClusterSchedulerBackend( } override def getDriverAttributes: Option[Map[String, String]] = { - var attributes: Option[Map[String, String]] = None try { val yarnConf = new YarnConfiguration(sc.hadoopConfiguration) val containerId = YarnSparkHadoopUtil.getContainerId @@ -91,7 +90,7 @@ private[spark] class YarnClusterSchedulerBackend( val user = Utils.getCurrentUserName() val httpScheme = if (yarnHttpPolicy == "HTTPS_ONLY") "https://" else "http://" - attributes = Some(Map( + Some(Map( "HTTP_SCHEME" -> httpScheme, "NODE_HTTP_ADDRESS" -> httpAddress, "CLUSTER_ID" -> clusterId.getOrElse(""), @@ -103,7 +102,7 @@ private[spark] class YarnClusterSchedulerBackend( case e: Exception => logInfo("Error while retrieving attributes on driver, so driver logs will not " + "be replaced with custom log pattern", e) + None } - attributes } } From 63b02f572ea631860cbb7157446ea8e91b2b22b4 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Wed, 23 Jan 2019 05:49:31 +0900 Subject: [PATCH 19/28] Fix more nits --- .../main/scala/org/apache/spark/internal/config/History.scala | 4 ++-- docs/monitoring.md | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/History.scala b/core/src/main/scala/org/apache/spark/internal/config/History.scala index e57ebccff8e20..1d73f01cb84d0 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/History.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/History.scala @@ -130,14 +130,14 @@ private[spark] object History { .doc("Specifies custom spark executor log url for supporting external log service instead of " + "using cluster managers' application log urls in the history server. Spark will support " + "some path variables via patterns which can vary on cluster manager. Please check the " + - "documentation for your each cluster manager to see which patterns are supported, if any. " + + "documentation for your cluster manager to see which patterns are supported, if any. " + "This configuration has no effect on a live application, it only affects the history server.") .stringConf .createOptional val APPLY_CUSTOM_EXECUTOR_LOG_URL_TO_INCOMPLETE_APP = ConfigBuilder("spark.history.custom.executor.log.url.applyIncompleteApplication") - .doc("Whether apply custom executor log url, as specified by " + + .doc("Whether to apply custom executor log url, as specified by " + "`spark.history.custom.executor.log.url`, to incomplete application as well. " + "Even if this is true, this still only affects the behavior of the history server, " + "not running spark applications.") diff --git a/docs/monitoring.md b/docs/monitoring.md index 369fbc43f5e57..63b71fcc6f269 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -254,7 +254,7 @@ Security options for the Spark History Server are covered more detail in the Specifies custom spark executor log url for supporting external log service instead of using cluster managers' application log urls in the history server. Spark will support some path variables via patterns - which can vary on cluster manager. Please check the documentation for your each cluster manager to + which can vary on cluster manager. Please check the documentation for your cluster manager to see which patterns are supported, if any. This configuration has no effect on a live application, it only affects the history server.

From 31d8e4e54e4ca1c6fa6b9c4850a5b37ea420e0c9 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Wed, 23 Jan 2019 11:54:54 +0900 Subject: [PATCH 20/28] Address review comments from @vanzin --- .../history/HistoryAppStatusStore.scala | 67 ++++++++++++------- .../history/FsHistoryProviderSuite.scala | 19 +++--- project/MimaExcludes.scala | 14 ++-- 3 files changed, 57 insertions(+), 43 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala index 2b4210c7007b0..624069f2691cc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala @@ -17,38 +17,42 @@ package org.apache.spark.deploy.history +import java.util.concurrent.atomic.AtomicBoolean + import scala.util.matching.Regex import org.apache.spark.SparkConf import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.History.{APPLY_CUSTOM_EXECUTOR_LOG_URL_TO_INCOMPLETE_APP, CUSTOM_EXECUTOR_LOG_URL} -import org.apache.spark.status.{AppStatusListener, AppStatusStore} +import org.apache.spark.internal.config.History._ +import org.apache.spark.status.AppStatusStore import org.apache.spark.status.api.v1 import org.apache.spark.status.api.v1.ExecutorSummary import org.apache.spark.util.kvstore.KVStore private[spark] class HistoryAppStatusStore( conf: SparkConf, - store: KVStore, - listener: Option[AppStatusListener] = None) - extends AppStatusStore(store, listener) with Logging { + store: KVStore) + extends AppStatusStore(store, None) with Logging { import HistoryAppStatusStore._ - private val logUrlPattern: Option[String] = conf.get(CUSTOM_EXECUTOR_LOG_URL) private val applyReplaceLogUrlToIncompleteApp: Boolean = conf.get(APPLY_CUSTOM_EXECUTOR_LOG_URL_TO_INCOMPLETE_APP) + private val logUrlPattern: Option[String] = { + if (isApplicationCompleted || applyReplaceLogUrlToIncompleteApp) { + conf.get(CUSTOM_EXECUTOR_LOG_URL) + } else { + None + } + } + + private var warnedForMissingAttributes = new AtomicBoolean(false) + override def executorList(activeOnly: Boolean): Seq[v1.ExecutorSummary] = { val execList = super.executorList(activeOnly) logUrlPattern match { - case Some(pattern) => - if (applyReplaceLogUrlToIncompleteApp || isApplicationCompleted) { - execList.map(replaceLogUrls(_, pattern)) - } else { - execList - } - + case Some(pattern) => execList.map(replaceLogUrls(_, pattern)) case None => execList } } @@ -56,13 +60,7 @@ private[spark] class HistoryAppStatusStore( override def executorSummary(executorId: String): v1.ExecutorSummary = { val execSummary = super.executorSummary(executorId) logUrlPattern match { - case Some(pattern) => - if (applyReplaceLogUrlToIncompleteApp || isApplicationCompleted) { - replaceLogUrls(execSummary, pattern) - } else { - execSummary - } - + case Some(pattern) => replaceLogUrls(execSummary, pattern) case None => execSummary } } @@ -75,6 +73,12 @@ private[spark] class HistoryAppStatusStore( private def replaceLogUrls(exec: ExecutorSummary, urlPattern: String): ExecutorSummary = { val attributes = exec.attributes + // Relation between pattern {{FILE_NAME}} and attribute {{LOG_FILES}} + // Given that HistoryAppStatusStore don't know which types of log files can be provided + // from resource manager, we require resource manager to provide available types of log + // files, which are encouraged to be same as types of log files provided in original log URLs. + // Once we get the list of log files, we need to expose them to end users as a pattern + // so that end users can compose custom log URL(s) including log file name(s). val allPatterns = CUSTOM_URL_PATTERN_REGEX.findAllMatchIn(urlPattern).map(_.group(1)).toSet val allPatternsExceptFileName = allPatterns.filter(_ != "FILE_NAME") val allAttributeKeys = attributes.keySet @@ -90,7 +94,7 @@ private[spark] class HistoryAppStatusStore( return exec } - val updatedUrl = allPatternsExceptFileName.foldLeft(urlPattern) { case(orig, patt) => + val updatedUrl = allPatternsExceptFileName.foldLeft(urlPattern) { case (orig, patt) => // we already checked the existence of attribute when comparing keys orig.replace(s"{{$patt}}", attributes(patt)) } @@ -104,7 +108,7 @@ private[spark] class HistoryAppStatusStore( Map("log" -> updatedUrl) } - exec.replaceExecutorLogs(newLogUrlMap) + replaceExecutorLogs(exec, newLogUrlMap) } private def logFailToRenewLogUrls( @@ -112,12 +116,25 @@ private[spark] class HistoryAppStatusStore( allPatterns: Set[String], allAttributes: Set[String]): Unit = { - logWarning(s"Fail to renew executor log urls: $reason. Required: $allPatterns / " + - s"available: $allAttributes. Failing back to show app's origin log urls.") + if (warnedForMissingAttributes.compareAndSet(false, true)) { + logWarning(s"Fail to renew executor log urls: $reason. Required: $allPatterns / " + + s"available: $allAttributes. Failing back to show app's original log urls.") + } + } + + private def replaceExecutorLogs(source: ExecutorSummary, newExecutorLogs: Map[String, String]) + : ExecutorSummary = { + new ExecutorSummary(source.id, source.hostPort, source.isActive, source.rddBlocks, + source.memoryUsed, source.diskUsed, source.totalCores, source.maxTasks, source.activeTasks, + source.failedTasks, source.completedTasks, source.totalTasks, source.totalDuration, + source.totalGCTime, source.totalInputBytes, source.totalShuffleRead, + source.totalShuffleWrite, source.isBlacklisted, source.maxMemory, source.addTime, + source.removeTime, source.removeReason, newExecutorLogs, source.memoryMetrics, + source.blacklistedInStages, source.peakMemoryMetrics, source.attributes) } } -object HistoryAppStatusStore { +private[spark] object HistoryAppStatusStore { val CUSTOM_URL_PATTERN_REGEX: Regex = "\\{\\{([A-Za-z0-9_\\-]+)\\}\\}".r } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 961c3a7e4844e..7d0e67fd40f40 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -292,7 +292,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc } } - test("Handling executor log url without renewing") { + test("log urls without customization") { val conf = createTestConf() val appId = "app1" val user = "user1" @@ -306,7 +306,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc testHandlingExecutorLogUrl(conf, expected) } - test("Handling executor log url with custom executor url - happy case") { + test("custom log urls, including FILE_NAME") { val conf = createTestConf() .set(CUSTOM_EXECUTOR_LOG_URL, "http://newhost:9999/logs/clusters/{{CLUSTER_ID}}" + "/users/{{USER}}/containers/{{CONTAINER_ID}}/{{FILE_NAME}}") @@ -332,8 +332,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc testHandlingExecutorLogUrl(conf, expected) } - test("Handling executor log url with custom executor url - happy case - " + - "pattern doesn't contain 'FILE_NAME'") { + test("custom log urls, excluding FILE_NAME") { val conf = createTestConf() .set(CUSTOM_EXECUTOR_LOG_URL, "http://newhost:9999/logs/clusters/{{CLUSTER_ID}}" + "/users/{{USER}}/containers/{{CONTAINER_ID}}") @@ -356,8 +355,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc testHandlingExecutorLogUrl(conf, expected) } - test("Handling executor log url with custom executor url - bad case - " + - "referring non-available attribute") { + test("custom log urls with invalid attribute") { // Here we are referring {{NON_EXISTING}} which is not available in attributes, // which Spark will fail back to provide origin log url with warning log. @@ -378,8 +376,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc testHandlingExecutorLogUrl(conf, expected) } - test("Handling executor log url with custom executor url - bad case - " + - "'FILE_NAME' is given for pattern but 'LOG_FILES' attribute is not available") { + test("custom log urls, LOG_FILES not available while FILE_NAME is specified") { // For this case Spark will fail back to provide origin log url with warning log. val conf = createTestConf() @@ -431,10 +428,10 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc var executorCount = 0 while (executors.hasNext) { val executor = executors.next() - val expectation = iterForExpectation.next() + val (expectedExecInfo, expectedLogs) = iterForExpectation.next() - executor.hostPort should startWith(expectation._1.executorHost) - executor.executorLogs should be(expectation._2) + executor.hostPort should startWith(expectedExecInfo.executorHost) + executor.executorLogs should be(expectedLogs) executorCount += 1 } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 3100d897375e4..8995c83cbbfa5 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -280,7 +280,13 @@ object MimaExcludes { // [SPARK-26616][MLlib] Expose document frequency in IDFModel ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.feature.IDFModel.this"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.mllib.feature.IDF#DocumentFrequencyAggregator.idf") + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.mllib.feature.IDF#DocumentFrequencyAggregator.idf"), + + // [SPARK-26311][CORE]New feature: apply custom log URL pattern for executor log URLs + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerApplicationStart.apply"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerApplicationStart.copy"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerApplicationStart.this"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.scheduler.SparkListenerApplicationStart$") ) // Exclude rules for 2.4.x @@ -1486,12 +1492,6 @@ object MimaExcludes { ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasLoss.org$apache$spark$ml$param$shared$HasLoss$_setter_$loss_="), ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasLoss.getLoss"), ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasLoss.loss") - ) ++ Seq( - // [SPARK-26311][CORE]New feature: apply custom log URL pattern for executor log URLs - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerApplicationStart.apply"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerApplicationStart.copy"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerApplicationStart.this"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.scheduler.SparkListenerApplicationStart$") ) } From d8574e9f43f49225f3f72c9e9a6d298cedc3d84c Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Thu, 24 Jan 2019 09:29:46 +0900 Subject: [PATCH 21/28] Change log level to INFO --- .../apache/spark/deploy/history/HistoryAppStatusStore.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala index 624069f2691cc..62f566d175603 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala @@ -47,7 +47,7 @@ private[spark] class HistoryAppStatusStore( } } - private var warnedForMissingAttributes = new AtomicBoolean(false) + private val informedForMissingAttributes = new AtomicBoolean(false) override def executorList(activeOnly: Boolean): Seq[v1.ExecutorSummary] = { val execList = super.executorList(activeOnly) @@ -116,8 +116,8 @@ private[spark] class HistoryAppStatusStore( allPatterns: Set[String], allAttributes: Set[String]): Unit = { - if (warnedForMissingAttributes.compareAndSet(false, true)) { - logWarning(s"Fail to renew executor log urls: $reason. Required: $allPatterns / " + + if (informedForMissingAttributes.compareAndSet(false, true)) { + logInfo(s"Fail to renew executor log urls: $reason. Required: $allPatterns / " + s"available: $allAttributes. Failing back to show app's original log urls.") } } From 29f33e470e725fbf64c832dcb943b7db0db1447f Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Fri, 25 Jan 2019 13:23:33 +0900 Subject: [PATCH 22/28] Addressed review comments from vanzin --- .../history/HistoryAppStatusStore.scala | 23 ++-- .../org/apache/spark/status/api/v1/api.scala | 11 +- .../application_list_json_expectation.json | 32 ----- .../completed_app_list_json_expectation.json | 16 --- .../limit_app_list_json_expectation.json | 33 +++--- .../minDate_app_list_json_expectation.json | 32 ----- .../minEndDate_app_list_json_expectation.json | 32 ----- .../running_app_list_json_expectation.json | 18 +-- .../application_1547723113049_0005 | 15 --- .../application_1547723113049_0006 | 12 -- .../history/FsHistoryProviderSuite.scala | 76 ++++++++---- .../deploy/history/HistoryServerSuite.scala | 66 +++-------- project/MimaExcludes.scala | 14 +-- .../spark/deploy/yarn/ExecutorRunnable.scala | 39 +++---- .../cluster/YarnClusterSchedulerBackend.scala | 71 +----------- .../spark/util/YarnExecutorHelper.scala | 109 ++++++++++++++++++ 16 files changed, 223 insertions(+), 376 deletions(-) delete mode 100644 core/src/test/resources/spark-events/application_1547723113049_0005 delete mode 100644 core/src/test/resources/spark-events/application_1547723113049_0006 create mode 100644 resource-managers/yarn/src/main/scala/org/apache/spark/util/YarnExecutorHelper.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala index 62f566d175603..6e9c2ed8a74b5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala @@ -26,7 +26,6 @@ import org.apache.spark.internal.Logging import org.apache.spark.internal.config.History._ import org.apache.spark.status.AppStatusStore import org.apache.spark.status.api.v1 -import org.apache.spark.status.api.v1.ExecutorSummary import org.apache.spark.util.kvstore.KVStore private[spark] class HistoryAppStatusStore( @@ -36,11 +35,10 @@ private[spark] class HistoryAppStatusStore( import HistoryAppStatusStore._ - private val applyReplaceLogUrlToIncompleteApp: Boolean = - conf.get(APPLY_CUSTOM_EXECUTOR_LOG_URL_TO_INCOMPLETE_APP) - private val logUrlPattern: Option[String] = { - if (isApplicationCompleted || applyReplaceLogUrlToIncompleteApp) { + val appInfo = super.applicationInfo() + val applicationCompleted = appInfo.attempts.nonEmpty && appInfo.attempts.head.completed + if (applicationCompleted || conf.get(APPLY_CUSTOM_EXECUTOR_LOG_URL_TO_INCOMPLETE_APP)) { conf.get(CUSTOM_EXECUTOR_LOG_URL) } else { None @@ -65,12 +63,7 @@ private[spark] class HistoryAppStatusStore( } } - private def isApplicationCompleted: Boolean = { - val appInfo = super.applicationInfo() - appInfo.attempts.nonEmpty && appInfo.attempts.head.completed - } - - private def replaceLogUrls(exec: ExecutorSummary, urlPattern: String): ExecutorSummary = { + private def replaceLogUrls(exec: v1.ExecutorSummary, urlPattern: String): v1.ExecutorSummary = { val attributes = exec.attributes // Relation between pattern {{FILE_NAME}} and attribute {{LOG_FILES}} @@ -115,16 +108,16 @@ private[spark] class HistoryAppStatusStore( reason: String, allPatterns: Set[String], allAttributes: Set[String]): Unit = { - if (informedForMissingAttributes.compareAndSet(false, true)) { logInfo(s"Fail to renew executor log urls: $reason. Required: $allPatterns / " + s"available: $allAttributes. Failing back to show app's original log urls.") } } - private def replaceExecutorLogs(source: ExecutorSummary, newExecutorLogs: Map[String, String]) - : ExecutorSummary = { - new ExecutorSummary(source.id, source.hostPort, source.isActive, source.rddBlocks, + private def replaceExecutorLogs( + source: v1.ExecutorSummary, + newExecutorLogs: Map[String, String]): v1.ExecutorSummary = { + new v1.ExecutorSummary(source.id, source.hostPort, source.isActive, source.rddBlocks, source.memoryUsed, source.diskUsed, source.totalCores, source.maxTasks, source.activeTasks, source.failedTasks, source.completedTasks, source.totalTasks, source.totalDuration, source.totalGCTime, source.totalInputBytes, source.totalShuffleRead, diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala index b6b544783ebbe..60b8bf8921814 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala @@ -107,16 +107,7 @@ class ExecutorSummary private[spark]( @JsonSerialize(using = classOf[ExecutorMetricsJsonSerializer]) @JsonDeserialize(using = classOf[ExecutorMetricsJsonDeserializer]) val peakMemoryMetrics: Option[ExecutorMetrics], - val attributes: Map[String, String]) { - - def replaceExecutorLogs(newExecutorLogs: Map[String, String]): ExecutorSummary = { - new ExecutorSummary(id, hostPort, isActive, rddBlocks, memoryUsed, diskUsed, totalCores, - maxTasks, activeTasks, failedTasks, completedTasks, totalTasks, totalDuration, - totalGCTime, totalInputBytes, totalShuffleRead, totalShuffleWrite, isBlacklisted, - maxMemory, addTime, removeTime, removeReason, newExecutorLogs, memoryMetrics, - blacklistedInStages, peakMemoryMetrics, attributes) - } -} + val attributes: Map[String, String]) class MemoryMetrics private[spark]( val usedOnHeapStorageMemory: Long, 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 20a8724e3c06d..0f0ccf9858a38 100644 --- a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json @@ -1,20 +1,4 @@ [ { - "id" : "application_1547723113049_0005", - "name" : "Spark SQL basic example", - "attempts" : [ { - "attemptId" : "1", - "startTime" : "2019-01-18T05:14:04.248GMT", - "endTime" : "2019-01-18T05:14:39.714GMT", - "lastUpdated" : "", - "duration" : 35466, - "sparkUser" : "spark", - "completed" : true, - "appSparkVersion" : "3.0.0-SNAPSHOT", - "startTimeEpoch" : 1547788444248, - "endTimeEpoch" : 1547788479714, - "lastUpdatedEpoch" : 0 - } ] -}, { "id" : "application_1538416563558_0014", "name" : "PythonBisectingKMeansExample", "attempts" : [ { @@ -220,20 +204,4 @@ "startTimeEpoch" : 1422981758277, "endTimeEpoch" : 1422981766912 } ] -}, { - "id" : "application_1547723113049_0006", - "name" : "Spark SQL basic example", - "attempts" : [ { - "attemptId" : "1", - "startTime" : "2019-01-18T05:14:14.200GMT", - "endTime" : "1969-12-31T23:59:59.999GMT", - "lastUpdated" : "", - "duration" : 0, - "sparkUser" : "spark", - "completed" : false, - "appSparkVersion" : "3.0.0-SNAPSHOT", - "startTimeEpoch" : 1547788454200, - "endTimeEpoch" : -1, - "lastUpdatedEpoch" : 0 - } ] } ] 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 7d6f1e7eb956f..e136a35a1e3a9 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,20 +1,4 @@ [ { - "id" : "application_1547723113049_0005", - "name" : "Spark SQL basic example", - "attempts" : [ { - "attemptId" : "1", - "startTime" : "2019-01-18T05:14:04.248GMT", - "endTime" : "2019-01-18T05:14:39.714GMT", - "lastUpdated" : "", - "duration" : 35466, - "sparkUser" : "spark", - "completed" : true, - "appSparkVersion" : "3.0.0-SNAPSHOT", - "startTimeEpoch" : 1547788444248, - "endTimeEpoch" : 1547788479714, - "lastUpdatedEpoch" : 0 - } ] -}, { "id" : "application_1538416563558_0014", "name" : "PythonBisectingKMeansExample", "attempts" : [ { diff --git a/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json index 0305df8c7c33b..0ef9377dcb08b 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,20 +1,4 @@ [ { - "id" : "application_1547723113049_0005", - "name" : "Spark SQL basic example", - "attempts" : [ { - "attemptId" : "1", - "startTime" : "2019-01-18T05:14:04.248GMT", - "endTime" : "2019-01-18T05:14:39.714GMT", - "lastUpdated" : "", - "duration" : 35466, - "sparkUser" : "spark", - "completed" : true, - "appSparkVersion" : "3.0.0-SNAPSHOT", - "startTimeEpoch" : 1547788444248, - "endTimeEpoch" : 1547788479714, - "lastUpdatedEpoch" : 0 - } ] -}, { "id" : "application_1538416563558_0014", "name" : "PythonBisectingKMeansExample", "attempts" : [ { @@ -44,4 +28,19 @@ "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 2c679a5bb3e05..ea9dc1b97afc8 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,20 +1,4 @@ [ { - "id" : "application_1547723113049_0005", - "name" : "Spark SQL basic example", - "attempts" : [ { - "attemptId" : "1", - "startTime" : "2019-01-18T05:14:04.248GMT", - "endTime" : "2019-01-18T05:14:39.714GMT", - "lastUpdated" : "", - "duration" : 35466, - "sparkUser" : "spark", - "completed" : true, - "appSparkVersion" : "3.0.0-SNAPSHOT", - "startTimeEpoch" : 1547788444248, - "endTimeEpoch" : 1547788479714, - "lastUpdatedEpoch" : 0 - } ] -}, { "id" : "application_1538416563558_0014", "name" : "PythonBisectingKMeansExample", "attempts" : [ { @@ -190,20 +174,4 @@ "startTimeEpoch" : 1425081758277, "endTimeEpoch" : 1425081766912 } ] -}, { - "id" : "application_1547723113049_0006", - "name" : "Spark SQL basic example", - "attempts" : [ { - "attemptId" : "1", - "startTime" : "2019-01-18T05:14:14.200GMT", - "endTime" : "1969-12-31T23:59:59.999GMT", - "lastUpdated" : "", - "duration" : 0, - "sparkUser" : "spark", - "completed" : false, - "appSparkVersion" : "3.0.0-SNAPSHOT", - "startTimeEpoch" : 1547788454200, - "endTimeEpoch" : -1, - "lastUpdatedEpoch" : 0 - } ] } ] 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 17b3e728e4848..2a77071a9ffd9 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,20 +1,4 @@ [ { - "id" : "application_1547723113049_0005", - "name" : "Spark SQL basic example", - "attempts" : [ { - "attemptId" : "1", - "startTime" : "2019-01-18T05:14:04.248GMT", - "endTime" : "2019-01-18T05:14:39.714GMT", - "lastUpdated" : "", - "duration" : 35466, - "sparkUser" : "spark", - "completed" : true, - "appSparkVersion" : "3.0.0-SNAPSHOT", - "startTimeEpoch" : 1547788444248, - "endTimeEpoch" : 1547788479714, - "lastUpdatedEpoch" : 0 - } ] -}, { "id" : "application_1538416563558_0014", "name" : "PythonBisectingKMeansExample", "attempts" : [ { @@ -147,20 +131,4 @@ "startTimeEpoch" : 1430917380880, "endTimeEpoch" : 1430917380890 } ] -}, { - "id" : "application_1547723113049_0006", - "name" : "Spark SQL basic example", - "attempts" : [ { - "attemptId" : "1", - "startTime" : "2019-01-18T05:14:14.200GMT", - "endTime" : "1969-12-31T23:59:59.999GMT", - "lastUpdated" : "", - "duration" : 0, - "sparkUser" : "spark", - "completed" : false, - "appSparkVersion" : "3.0.0-SNAPSHOT", - "startTimeEpoch" : 1547788454200, - "endTimeEpoch" : -1, - "lastUpdatedEpoch" : 0 - } ] } ] diff --git a/core/src/test/resources/HistoryServerExpectations/running_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/running_app_list_json_expectation.json index b766580def925..1e3ec7217afba 100644 --- a/core/src/test/resources/HistoryServerExpectations/running_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/running_app_list_json_expectation.json @@ -1,17 +1 @@ -[ { - "id" : "application_1547723113049_0006", - "name" : "Spark SQL basic example", - "attempts" : [ { - "attemptId" : "1", - "startTime" : "2019-01-18T05:14:14.200GMT", - "endTime" : "1969-12-31T23:59:59.999GMT", - "lastUpdated" : "", - "duration" : 0, - "sparkUser" : "spark", - "completed" : false, - "appSparkVersion" : "3.0.0-SNAPSHOT", - "startTimeEpoch" : 1547788454200, - "endTimeEpoch" : -1, - "lastUpdatedEpoch" : 0 - } ] -} ] +[ ] diff --git a/core/src/test/resources/spark-events/application_1547723113049_0005 b/core/src/test/resources/spark-events/application_1547723113049_0005 deleted file mode 100644 index 41ca28d8ac436..0000000000000 --- a/core/src/test/resources/spark-events/application_1547723113049_0005 +++ /dev/null @@ -1,15 +0,0 @@ -{"Event":"SparkListenerLogStart","Spark Version":"3.0.0-SNAPSHOT"} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"node2","Port":40675},"Maximum Memory":428762726,"Timestamp":1547788445481,"Maximum Onheap Memory":428762726,"Maximum Offheap Memory":0} -{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre","Java Version":"1.8.0_161 (Oracle Corporation)","Scala Version":"version 2.12.8"},"Spark Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.scheduler.mode":"FIFO","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"},"System Properties":{"java.io.tmpdir":"/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_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/yarn","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/amd64","user.dir":"/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","java.library.path":"/usr/java/packages/lib/amd64:/usr/lib64:/lib64:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.161-b14","jetty.git.hash":"27208684755d94a92186989f695db2d7b21ebc51","java.endorsed.dirs":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/endorsed","java.runtime.version":"1.8.0_161-b14","java.vm.info":"mixed mode","java.ext.dirs":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"OpenJDK Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/resources.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/rt.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/sunrsasign.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/jsse.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/jce.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/charsets.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/jfr.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/classes","file.encoding":"UTF-8","user.timezone":"UTC","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"4.18.5-1.el7.elrepo.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":"yarn","hdp.version":"2.6.5.1002-73","java.vm.name":"OpenJDK 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.yarn.ApplicationMaster --class org.apache.spark.examples.sql.SparkSQLExample --jar file:/home/spark/spark-3.0.0-snapshot-spark-26311-v2/examples/jars/spark-examples_2.12-3.0.0-SNAPSHOT.jar --properties-file /hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_conf__/__spark_conf__.properties --dist-cache-conf /hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_conf__/__spark_dist_cache__.properties","java.home":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre","java.version":"1.8.0_161","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/hadoop-yarn-client/lib/commons-digester-1.8.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-applications-distributedshell-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-compiler-3.0.11.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-hadoop-bundle-1.6.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/lz4-java-1.5.0.jar":"System Classpath","/hadoop/lib/commons-digester-1.8.jar":"System Classpath","/hadoop-yarn-client/lib/netty-3.6.2.Final.jar":"System Classpath","/hadoop/lib/curator-client-2.7.1.jar":"System Classpath","/hadoop/lib/log4j-1.2.17.jar":"System Classpath","/hadoop/lib/paranamer-2.3.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-codec-1.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hk2-api-2.4.0-b34.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/orc-core-1.5.4-nohive.jar":"System Classpath","/hadoop-yarn-client/lib/curator-client-2.7.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop/hadoop-common.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-cli-1.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-hdfs-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/aopalliance-1.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-guava-2.22.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/stax-api-1.0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/snappy-java-1.1.7.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/univocity-parsers-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/azure-storage-5.4.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/netty-all-4.1.30.Final.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-container-servlet-core-2.22.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/macro-compat_2.12-1.1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-httpclient-3.1.jar":"System Classpath","/hadoop/lib/httpcore-4.4.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/hadoop/lib/java-xmlbuilder-0.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jodd-core-3.5.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/breeze-macros_2.12-0.13.2.jar":"System Classpath","/hadoop-yarn-client/lib/zookeeper-3.4.6.jar":"System Classpath","/hadoop/lib/curator-framework-2.7.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jul-to-slf4j-1.7.16.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/xz-1.5.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/antlr-2.7.7.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/json4s-scalap_2.12-3.5.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/apache-log4j-extras-1.2.17.jar":"System Classpath","/hadoop-hdfs-client/lib/htrace-core-3.1.0-incubating.jar":"System Classpath","/hadoop/hadoop-azure.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-nodemanager.jar":"System Classpath","/hadoop-yarn-client/lib/servlet-api-2.5.jar":"System Classpath","/hadoop-yarn-client/lib/guava-11.0.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/arpack_combined_all-0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/arrow-format-0.10.0.jar":"System Classpath","/hadoop-yarn-client/lib/guice-3.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/scala-library-2.12.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-container-servlet-2.22.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/xercesImpl-2.9.1.jar":"System Classpath","/hadoop/hadoop-azure-datalake-2.7.3.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-common.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/yarn/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-auth-2.7.4.jar":"System Classpath","/hadoop/lib/activation-1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jline-2.14.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/yarn/lib/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hive-metastore-1.2.1.spark2.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-applicationhistoryservice.jar":"System Classpath","/hadoop/lib/ojdbc6.jar":"System Classpath","/hadoop/lib/commons-collections-3.2.2.jar":"System Classpath","/hadoop-hdfs-client/lib/servlet-api-2.5.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/joda-time-2.9.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-annotations-2.9.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-mllib-local_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/leveldbjni-all-1.8.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-common-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/json4s-jackson_2.12-3.5.3.jar":"System Classpath","/hadoop/conf":"System Classpath","/hadoop/hadoop-common-2.7.3-tests.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-timeline-pluginstorage-2.7.3.jar":"System Classpath","/hadoop/lib/junit-4.11.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs-2.7.3-tests.jar":"System Classpath","/hadoop-yarn-client/lib/httpcore-4.4.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-unsafe_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/metrics-jvm-3.1.5.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/objenesis-2.5.1.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-nodemanager-2.7.3.jar":"System Classpath","/hadoop/hadoop-common-tests.jar":"System Classpath","/hadoop-hdfs-client/lib/jackson-databind-2.2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-graphx_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-hdfs-client/lib/asm-3.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/kryo-shaded-4.0.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/compress-lzf-1.0.3.jar":"System Classpath","/hadoop/lib/azure-storage-5.4.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/avro-mapred-1.8.2-hadoop2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/scala-xml_2.12-1.0.5.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-applications-distributedshell.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jaxb-api-2.2.2.jar":"System Classpath","/hadoop/lib/commons-net-3.1.jar":"System Classpath","/hadoop-yarn-client/lib/java-xmlbuilder-0.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/paranamer-2.8.jar":"System Classpath","/hadoop/lib/commons-logging-1.1.3.jar":"System Classpath","/hadoop-yarn-client/lib/log4j-1.2.17.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/mapreduce/*":"System Classpath","/hadoop/lib/commons-codec-1.4.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs-nfs-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/commons-lang3-3.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-math3-3.4.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-network-common_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jetty-6.1.26.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/javolution-5.5.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jcl-over-slf4j-1.7.16.jar":"System Classpath","/hadoop/lib/slf4j-api-1.7.10.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-common-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-core-asl-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hk2-utils-2.4.0-b34.jar":"System Classpath","/hadoop/lib/mockito-all-1.8.5.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-client.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/common/lib/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hppc-0.7.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/shapeless_2.12-2.3.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-mapreduce-client-common-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/log4j-1.2.17.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-module-scala_2.12-2.9.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-module-paranamer-2.9.6.jar":"System Classpath","/hadoop-hdfs-client/lib/jetty-6.1.26.hwx.jar":"System Classpath","/hadoop/hadoop-aws.jar":"System Classpath","/hadoop/lib/commons-cli-1.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-codec-1.10.jar":"System Classpath","/hadoop/lib/commons-compress-1.4.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-server-2.22.2.jar":"System Classpath","/hadoop/lib/avro-1.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-xc-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-dbcp-1.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-beanutils-1.7.0.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-jaxrs-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/api-asn1-api-1.0.0-M20.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-client-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/javax.inject-2.4.0-b34.jar":"System Classpath","/hadoop/gcs-connector-1.8.1-shaded.jar":"System Classpath","/hadoop-hdfs-client/lib/jersey-core-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/metrics-json-3.1.5.jar":"System Classpath","/hadoop/lib/jackson-core-2.2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jpam-1.1.jar":"System Classpath","/hadoop/lib/jersey-core-1.9.jar":"System Classpath","/hadoop/lib/snappy-java-1.0.4.1.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-lang-2.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/slf4j-log4j12-1.7.16.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/osgi-resource-locator-1.0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/api-asn1-api-1.0.0-M20.jar":"System Classpath","/hadoop-yarn-client/lib/nimbus-jose-jwt-4.41.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-databind-2.9.6.jar":"System Classpath","/hadoop-hdfs-client/lib/xml-apis-1.3.04.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-client-2.22.2.jar":"System Classpath","/hadoop/hadoop-nfs-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hive-jdbc-1.2.1.spark2.jar":"System Classpath","/hadoop/lib/jackson-databind-2.2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/eigenbase-properties-1.1.5.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/machinist_2.12-0.6.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/tools/lib/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-jackson-1.10.0.jar":"System Classpath","/hadoop-yarn-client/lib/jetty-util-6.1.26.hwx.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-digester-1.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/calcite-avatica-1.2.0-incubating.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/oro-2.0.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hive-cli-1.2.1.spark2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-hive-thriftserver_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-applications-unmanaged-am-launcher-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/JavaEWAH-0.3.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/super-csv-2.2.0.jar":"System Classpath","/hadoop/lib/ranger-yarn-plugin-shim-0.7.0.jar":"System Classpath","/hadoop-yarn-client/lib/jetty-6.1.26.hwx.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/metrics-core-3.1.5.jar":"System Classpath","/hadoop-yarn-client/lib/commons-compress-1.4.1.jar":"System Classpath","/hadoop-hdfs-client/lib/netty-all-4.0.52.Final.jar":"System Classpath","/hadoop/hadoop-auth.jar":"System Classpath","/hadoop-yarn-client/lib/javax.inject-1.jar":"System Classpath","/hadoop/lib/jettison-1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jetty-util-6.1.26.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-network-shuffle_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/lib/jackson-jaxrs-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/json-smart-1.3.1.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-api-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/commons-beanutils-1.7.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-streaming_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/lib/commons-io-2.4.jar":"System Classpath","/hadoop/lib/commons-beanutils-1.7.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/snappy-0.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jta-1.1.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-resourcemanager-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/protobuf-java-2.5.0.jar":"System Classpath","/hadoop-yarn-client/lib/curator-framework-2.7.1.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-daemon-1.0.13.jar":"System Classpath","/hadoop/lib/hadoop-lzo-0.6.0-sources.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-common.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/breeze_2.12-0.13.2.jar":"System Classpath","/hadoop/lib/jsp-api-2.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/orc-mapreduce-1.5.4-nohive.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-mapreduce-client-core-2.7.4.jar":"System Classpath","/hadoop/lib/htrace-core-3.1.0-incubating.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-encoding-1.10.0.jar":"System Classpath","/hadoop/hadoop-auth-2.7.3.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-applicationhistoryservice-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/jettison-1.1.jar":"System Classpath","/hadoop-yarn-client/lib/commons-codec-1.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-mapreduce-client-shuffle-2.7.4.jar":"System Classpath","/hadoop/lib/curator-recipes-2.7.1.jar":"System Classpath","/hadoop/lib/jetty-util-6.1.26.hwx.jar":"System Classpath","/hadoop-hdfs-client/lib/log4j-1.2.17.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/activation-1.1.1.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/jaxb-api-2.2.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/opencsv-2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-collections-3.2.2.jar":"System Classpath","/hadoop/lib/guava-11.0.2.jar":"System Classpath","/hadoop-yarn-client/lib/commons-net-3.1.jar":"System Classpath","/hadoop/lib/commons-beanutils-core-1.8.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/py4j-0.10.8.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/json4s-ast_2.12-3.5.3.jar":"System Classpath","/hadoop-yarn-client/lib/metrics-core-3.0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/pyrolite-4.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/chill_2.12-0.9.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/libfb303-0.9.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/javax.inject-1.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs.jar":"System Classpath","/ext/hadoop/*":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-web-proxy-2.7.3.jar":"System Classpath","/hadoop/lib/aws-java-sdk-core-1.10.6.jar":"System Classpath","/hadoop-yarn-client/lib/commons-configuration-1.6.jar":"System Classpath","/hadoop/lib/zookeeper-3.4.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-logging-1.1.3.jar":"System Classpath","/hadoop-hdfs-client/lib/jsr305-3.0.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/stringtemplate-3.2.1.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-resourcemanager.jar":"System Classpath","/hadoop/lib/netty-3.6.2.Final.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/zstd-jni-1.3.2-2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jsp-api-2.1.jar":"System Classpath","/hadoop-yarn-client/lib/jaxb-impl-2.2.3-1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-tags_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hive-beeline-1.2.1.spark2.jar":"System Classpath","/hadoop-hdfs-client/lib/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/snappy-java-1.0.4.1.jar":"System Classpath","/hadoop-yarn-client/lib/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs-nfs.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/hadoop/hadoop-aws-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/jsr305-3.0.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/datanucleus-rdbms-3.2.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/libthrift-0.9.3.jar":"System Classpath","/hadoop/lib/xz-1.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/json4s-core_2.12-3.5.3.jar":"System Classpath","/hadoop-yarn-client/lib/jersey-guice-1.9.jar":"System Classpath","/hadoop-yarn-client/lib/activation-1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-jaxrs-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/hdfs/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/mapreduce/lib/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-pool-1.5.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/arrow-memory-0.10.0.jar":"System Classpath","/hadoop-hdfs-client/lib/jetty-util-6.1.26.hwx.jar":"System Classpath","/hadoop/lib/ranger-plugin-classloader-0.7.0.jar":"System Classpath","/hadoop/lib/aws-java-sdk-kms-1.10.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-repl_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-yarn-client/lib/jersey-json-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-yarn_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-yarn-client/lib/xz-1.0.jar":"System Classpath","/hadoop/hadoop-common-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jetty-sslengine-6.1.26.jar":"System Classpath","/hadoop/lib/jaxb-api-2.2.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/metrics-graphite-3.1.5.jar":"System Classpath","/hadoop/lib/jcip-annotations-1.0-1.jar":"System Classpath","/hadoop/lib/nimbus-jose-jwt-4.41.1.jar":"System Classpath","/hadoop/lib/hadoop-lzo-0.6.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hk2-locator-2.4.0-b34.jar":"System Classpath","/hadoop/lib/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-logging-1.1.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jsr305-3.0.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-hive_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/RoaringBitmap-0.5.11.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/calcite-linq4j-1.2.0-incubating.jar":"System Classpath","/hadoop-yarn-client/lib/commons-beanutils-core-1.8.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-lang-2.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/derby-10.12.1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-hadoop-1.10.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/stax-api-1.0-2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/scala-reflect-2.12.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/calcite-core-1.2.0-incubating.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_conf__":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-format-2.4.0.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-sharedcachemanager-2.7.3.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-web-proxy.jar":"System Classpath","/hadoop-hdfs-client/lib/xercesImpl-2.9.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/netty-3.9.9.Final.jar":"System Classpath","/hadoop/lib/xmlenc-0.52.jar":"System Classpath","/hadoop/hadoop-annotations-2.7.3.jar":"System Classpath","/hadoop-hdfs-client/lib/protobuf-java-2.5.0.jar":"System Classpath","/hadoop-yarn-client/lib/jersey-core-1.9.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-registry-2.7.3.jar":"System Classpath","/hadoop-hdfs-client/lib/okhttp-2.7.5.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-yarn-common-2.7.4.jar":"System Classpath","/hadoop-yarn-client/lib/leveldbjni-all-1.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-crypto-1.0.0.jar":"System Classpath","/hadoop/lib/jetty-6.1.26.hwx.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/aopalliance-repackaged-2.4.0-b34.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_conf__/__hadoop_conf__":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-compress-1.8.1.jar":"System Classpath","/hadoop/lib/jackson-xc-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/gson-2.2.4.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-api.jar":"System Classpath","/hadoop/lib/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs-tests.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-io-2.4.jar":"System Classpath","/hadoop/hadoop-nfs.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-yarn-api-2.7.4.jar":"System Classpath","/hadoop/lib/json-smart-1.3.1.jar":"System Classpath","/hadoop-yarn-client/lib/commons-lang-2.6.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/azure-keyvault-core-0.8.0.jar":"System Classpath","/hadoop-yarn-client/lib/fst-2.24.jar":"System Classpath","/hadoop-yarn-client/lib/htrace-core-3.1.0-incubating.jar":"System Classpath","/hadoop/hadoop-azure-datalake.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-common-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hive-exec-1.2.1.spark2.jar":"System Classpath","/hadoop-hdfs-client/lib/jersey-server-1.9.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-sharedcachemanager.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-launcher_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/datanucleus-core-3.2.10.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/zookeeper-3.4.6.jar":"System Classpath","/hadoop-yarn-client/lib/commons-math3-3.1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/api-util-1.0.0-M20.jar":"System Classpath","/hadoop/lib/azure-keyvault-core-0.8.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-common-2.22.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jtransforms-2.4.0.jar":"System Classpath","/hadoop/lib/asm-3.2.jar":"System Classpath","/hadoop/lib/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/javax.ws.rs-api-2.0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/xmlenc-0.52.jar":"System Classpath","/hadoop/lib/jsch-0.1.54.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-timeline-pluginstorage.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-annotations-2.2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/hdfs/lib/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/httpclient-4.5.6.jar":"System Classpath","/hadoop-hdfs-client/lib/xmlenc-0.52.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-annotations-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/antlr-runtime-3.4.jar":"System Classpath","/etc/hadoop/conf/secure":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/avro-ipc-1.8.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/janino-3.0.11.jar":"System Classpath","/hadoop/lib/api-util-1.0.0-M20.jar":"System Classpath","/hadoop-yarn-client/lib/xmlenc-0.52.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/slf4j-api-1.7.16.jar":"System Classpath","/hadoop/lib/ranger-hdfs-plugin-shim-0.7.0.jar":"System Classpath","/hadoop-hdfs-client/lib/guava-11.0.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/chill-java-0.9.3.jar":"System Classpath","/hadoop-yarn-client/lib/aopalliance-1.0.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-applications-unmanaged-am-launcher.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/htrace-core-3.1.0-incubating.jar":"System Classpath","/hadoop/lib/jersey-json-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/guava-14.0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-yarn-server-web-proxy-2.7.4.jar":"System Classpath","/hadoop/lib/jetty-sslengine-6.1.26.hwx.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/ST4-4.0.4.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-client-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-sketch_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-yarn-client/lib/httpclient-4.5.2.jar":"System Classpath","/hadoop/lib/httpclient-4.5.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/antlr4-runtime-4.7.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/arrow-vector-0.10.0.jar":"System Classpath","/hadoop-yarn-client/lib/jersey-server-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-yarn-client-2.7.4.jar":"System Classpath","/hadoop-yarn-client/lib/jsch-0.1.54.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/curator-client-2.7.1.jar":"System Classpath","/hadoop-yarn-client/lib/stax-api-1.0-2.jar":"System Classpath","/hadoop-yarn-client/lib/commons-cli-1.2.jar":"System Classpath","/hadoop/lib/servlet-api-2.5.jar":"System Classpath","/hadoop-yarn-client/lib/commons-collections-3.2.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-net-3.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-media-jaxb-2.22.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-mapreduce-client-app-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-core_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-catalyst_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-kvstore_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/lib/joda-time-2.9.4.jar":"System Classpath","/hadoop-yarn-client/lib/jersey-client-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-lang3-3.8.1.jar":"System Classpath","/hadoop/lib/slf4j-log4j12-1.7.10.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/bonecp-0.8.0.RELEASE.jar":"System Classpath","/hadoop-yarn-client/lib/jetty-sslengine-6.1.26.hwx.jar":"System Classpath","/hadoop-yarn-client/lib/commons-logging-1.1.3.jar":"System Classpath","/hadoop/lib/jets3t-0.9.0.jar":"System Classpath","/hadoop/lib/hadoop-lzo-0.6.0-javadoc.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/orc-shims-1.5.4.jar":"System Classpath","/hadoop-hdfs-client/lib/okio-1.6.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-column-1.10.0.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-core-asl-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/jsp-api-2.1.jar":"System Classpath","/hadoop/lib/aws-java-sdk-s3-1.10.6.jar":"System Classpath","/hadoop/azure-data-lake-store-sdk-2.2.5.jar":"System Classpath","/hadoop-yarn-client/lib/zookeeper-3.4.6-tests.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/stream-2.7.0.jar":"System Classpath","/hadoop/hadoop-azure-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/jcip-annotations-1.0-1.jar":"System Classpath","/hadoop-yarn-client/lib/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/common/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/aircompressor-0.10.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/httpcore-4.4.10.jar":"System Classpath","/hadoop-yarn-client/lib/objenesis-2.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/avro-1.8.2.jar":"System Classpath","/hadoop-yarn-client/lib/jets3t-0.9.0.jar":"System Classpath","/hadoop-hdfs-client/lib/jackson-core-2.2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-configuration-1.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-mapreduce-client-jobclient-2.7.4.jar":"System Classpath","/hadoop-yarn-client/lib/commons-io-2.4.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-tests.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/guice-3.0.jar":"System Classpath","/hadoop/lib/jsr305-3.0.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/javassist-3.18.1-GA.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/curator-framework-2.7.1.jar":"System Classpath","/hadoop-yarn-client/lib/avro-1.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/scala-compiler-2.12.8.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-registry.jar":"System Classpath","/hadoop/lib/stax-api-1.0-2.jar":"System Classpath","/hadoop-yarn-client/lib/guice-servlet-3.0.jar":"System Classpath","/hadoop/lib/commons-math3-3.1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/javax.annotation-api-1.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jdo-api-3.0.1.jar":"System Classpath","/hadoop-hdfs-client/lib/jackson-core-asl-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-sql_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-yarn-server-common-2.7.4.jar":"System Classpath","/hadoop-yarn-client/lib/api-util-1.0.0-M20.jar":"System Classpath","/hadoop/lib/api-asn1-api-1.0.0-M20.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/minlog-1.3.0.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-xc-1.9.13.jar":"System Classpath","/hadoop-hdfs-client/lib/netty-3.6.2.Final.jar":"System Classpath","/hadoop/lib/hamcrest-core-1.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/xbean-asm7-shaded-4.12.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-core-2.2.3.jar":"System Classpath","/hadoop/lib/jaxb-impl-2.2.3-1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/scala-parser-combinators_2.12-1.1.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-io-2.4.jar":"System Classpath","/hadoop/lib/commons-lang-2.6.jar":"System Classpath","/hadoop/lib/commons-lang3-3.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spire_2.12-0.13.0.jar":"System Classpath","/hadoop-hdfs-client/lib/jackson-annotations-2.2.3.jar":"System Classpath","/hadoop/lib/jackson-core-asl-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/javax.servlet-api-3.1.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/core-1.1.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-cli-1.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/curator-recipes-2.7.1.jar":"System Classpath","/hadoop/lib/jersey-server-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spire-macros_2.12-0.13.0.jar":"System Classpath","/hadoop-yarn-client/lib/javassist-3.18.1-GA.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/validation-api-1.1.0.Final.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/protobuf-java-2.5.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-beanutils-core-1.8.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/gson-2.2.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/ivy-2.4.0.jar":"System Classpath","/hadoop-yarn-client/lib/paranamer-2.3.jar":"System Classpath","/hadoop/lib/gson-2.2.4.jar":"System Classpath","/hadoop/lib/jackson-annotations-2.2.3.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-databind-2.2.3.jar":"System Classpath","/hadoop-hdfs-client/lib/leveldbjni-all-1.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/guice-servlet-3.0.jar":"System Classpath","/hadoop/hadoop-annotations.jar":"System Classpath","/hadoop-yarn-client/lib/curator-recipes-2.7.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/flatbuffers-1.2.0-3f79e055.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-mllib_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-tests-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-common-1.10.0.jar":"System Classpath","/hadoop-yarn-client/lib/asm-3.2.jar":"System Classpath","/hadoop/lib/protobuf-java-2.5.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-core-2.9.6.jar":"System Classpath","/hadoop/lib/commons-configuration-1.6.jar":"System Classpath"}} -{"Event":"SparkListenerApplicationStart","App Name":"Spark SQL basic example","App ID":"application_1547723113049_0005","Timestamp":1547788444248,"User":"spark","App Attempt ID":"1","Driver Logs":{"stdout":"http://node2:8042/node/containerlogs/container_e01_1547723113049_0005_01_000001/spark/stdout?start=-4096","stderr":"http://node2:8042/node/containerlogs/container_e01_1547723113049_0005_01_000001/spark/stderr?start=-4096"},"Driver Attributes":{"NODE_HTTP_ADDRESS":"node2:8042","USER":"spark","LOG_FILES":"stderr,stdout","CLUSTER_ID":"yarn-cluster","HTTP_SCHEME":"http://","CONTAINER_ID":"container_e01_1547723113049_0005_01_000001"}} -{"Event":"SparkListenerExecutorAdded","Timestamp":1547788454209,"Executor ID":"1","Executor Info":{"Host":"node3","Total Cores":1,"Log Urls":{"stdout":"http://node3:8042/node/containerlogs/container_e01_1547723113049_0005_01_000002/spark/stdout?start=-4096","stderr":"http://node3:8042/node/containerlogs/container_e01_1547723113049_0005_01_000002/spark/stderr?start=-4096"},"Attributes":{"NODE_HTTP_ADDRESS":"node3:8042","USER":"spark","LOG_FILES":"stderr,stdout","CLUSTER_ID":"yarn-cluster","HTTP_SCHEME":"http://","CONTAINER_ID":"container_e01_1547723113049_0005_01_000002"}}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"node3","Port":35087},"Maximum Memory":384093388,"Timestamp":1547788454413,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} -{"Event":"SparkListenerExecutorAdded","Timestamp":1547788455343,"Executor ID":"2","Executor Info":{"Host":"node4","Total Cores":1,"Log Urls":{"stdout":"http://node4:8042/node/containerlogs/container_e01_1547723113049_0005_01_000003/spark/stdout?start=-4096","stderr":"http://node4:8042/node/containerlogs/container_e01_1547723113049_0005_01_000003/spark/stderr?start=-4096"},"Attributes":{"NODE_HTTP_ADDRESS":"node4:8042","USER":"spark","LOG_FILES":"stderr,stdout","CLUSTER_ID":"yarn-cluster","HTTP_SCHEME":"http://","CONTAINER_ID":"container_e01_1547723113049_0005_01_000003"}}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"2","Host":"node4","Port":44875},"Maximum Memory":384093388,"Timestamp":1547788455624,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} -{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1547788459293,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"json at SparkSQLExample.scala:59","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"mapPartitions\"}","Callsite":"json at SparkSQLExample.scala:59","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"json at SparkSQLExample.scala:59","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"json at SparkSQLExample.scala:59","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameReader.json(DataFrameReader.scala:331)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:59)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[0],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"json at SparkSQLExample.scala:59","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"mapPartitions\"}","Callsite":"json at SparkSQLExample.scala:59","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"json at SparkSQLExample.scala:59","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"json at SparkSQLExample.scala:59","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameReader.json(DataFrameReader.scala:331)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:59)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788459321,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0005","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0005/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0005","spark.app.id":"application_1547723113049_0005","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1547788459627,"Executor ID":"1","Host":"node3","Locality":"NODE_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":1547788459627,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788463652,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"duration total (min, med, max)","Update":"1582","Value":"1581","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":0,"Name":"number of output rows","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"internal.metrics.input.recordsRead","Update":3,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":26,"Name":"internal.metrics.input.bytesRead","Update":73,"Value":73,"Internal":true,"Count Failed Values":true},{"ID":11,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.jvmGCTime","Update":159,"Value":159,"Internal":true,"Count Failed Values":true},{"ID":9,"Name":"internal.metrics.resultSize","Update":1963,"Value":1963,"Internal":true,"Count Failed Values":true},{"ID":8,"Name":"internal.metrics.executorCpuTime","Update":2225128499,"Value":2225128499,"Internal":true,"Count Failed Values":true},{"ID":7,"Name":"internal.metrics.executorRunTime","Update":2291,"Value":2291,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1171850277,"Value":1171850277,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.executorDeserializeTime","Update":1431,"Value":1431,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1431,"Executor Deserialize CPU Time":1171850277,"Executor Run Time":2291,"Executor CPU Time":2225128499,"Result Size":1963,"JVM GC Time":159,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":73,"Records Read":3},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} -{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"json at SparkSQLExample.scala:59","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"mapPartitions\"}","Callsite":"json at SparkSQLExample.scala:59","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"json at SparkSQLExample.scala:59","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"json at SparkSQLExample.scala:59","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameReader.json(DataFrameReader.scala:331)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:59)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788459321,"Completion Time":1547788463667,"Accumulables":[{"ID":26,"Name":"internal.metrics.input.bytesRead","Value":73,"Internal":true,"Count Failed Values":true},{"ID":8,"Name":"internal.metrics.executorCpuTime","Value":2225128499,"Internal":true,"Count Failed Values":true},{"ID":11,"Name":"internal.metrics.resultSerializationTime","Value":2,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.executorDeserializeTime","Value":1431,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"duration total (min, med, max)","Value":"1581","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":7,"Name":"internal.metrics.executorRunTime","Value":2291,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.jvmGCTime","Value":159,"Internal":true,"Count Failed Values":true},{"ID":27,"Name":"internal.metrics.input.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":9,"Name":"internal.metrics.resultSize","Value":1963,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.executorDeserializeCpuTime","Value":1171850277,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"number of output rows","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"}]}} -{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1547788463673,"Job Result":{"Result":"JobSucceeded"}} -{"Event":"SparkListenerApplicationEnd","Timestamp":1547788479714} diff --git a/core/src/test/resources/spark-events/application_1547723113049_0006 b/core/src/test/resources/spark-events/application_1547723113049_0006 deleted file mode 100644 index 12124f4f4a710..0000000000000 --- a/core/src/test/resources/spark-events/application_1547723113049_0006 +++ /dev/null @@ -1,12 +0,0 @@ -{"Event":"SparkListenerLogStart","Spark Version":"3.0.0-SNAPSHOT"} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"node2","Port":40675},"Maximum Memory":428762726,"Timestamp":1547788445481,"Maximum Onheap Memory":428762726,"Maximum Offheap Memory":0} -{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre","Java Version":"1.8.0_161 (Oracle Corporation)","Scala Version":"version 2.12.8"},"Spark Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0006","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.scheduler.mode":"FIFO","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0006","spark.app.id":"application_1547723113049_0006","spark.hadoop.yarn.timeline-service.enabled":"false"},"System Properties":{"java.io.tmpdir":"/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_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/yarn","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/amd64","user.dir":"/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001","java.library.path":"/usr/java/packages/lib/amd64:/usr/lib64:/lib64:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.161-b14","jetty.git.hash":"27208684755d94a92186989f695db2d7b21ebc51","java.endorsed.dirs":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/endorsed","java.runtime.version":"1.8.0_161-b14","java.vm.info":"mixed mode","java.ext.dirs":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"OpenJDK Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/resources.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/rt.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/sunrsasign.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/jsse.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/jce.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/charsets.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/lib/jfr.jar:/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre/classes","file.encoding":"UTF-8","user.timezone":"UTC","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"4.18.5-1.el7.elrepo.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":"yarn","hdp.version":"2.6.5.1002-73","java.vm.name":"OpenJDK 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.yarn.ApplicationMaster --class org.apache.spark.examples.sql.SparkSQLExample --jar file:/home/spark/spark-3.0.0-snapshot-spark-26311-v2/examples/jars/spark-examples_2.12-3.0.0-SNAPSHOT.jar --properties-file /hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_conf__/__spark_conf__.properties --dist-cache-conf /hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_conf__/__spark_dist_cache__.properties","java.home":"/usr/lib/jvm/java-1.8.0-openjdk-1.8.0.161-2.b14.el7.x86_64/jre","java.version":"1.8.0_161","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/hadoop-yarn-client/lib/commons-digester-1.8.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-applications-distributedshell-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-compiler-3.0.11.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-hadoop-bundle-1.6.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/lz4-java-1.5.0.jar":"System Classpath","/hadoop/lib/commons-digester-1.8.jar":"System Classpath","/hadoop-yarn-client/lib/netty-3.6.2.Final.jar":"System Classpath","/hadoop/lib/curator-client-2.7.1.jar":"System Classpath","/hadoop/lib/log4j-1.2.17.jar":"System Classpath","/hadoop/lib/paranamer-2.3.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-codec-1.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hk2-api-2.4.0-b34.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/orc-core-1.5.4-nohive.jar":"System Classpath","/hadoop-yarn-client/lib/curator-client-2.7.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop/hadoop-common.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-cli-1.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-hdfs-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/aopalliance-1.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-guava-2.22.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/stax-api-1.0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/snappy-java-1.1.7.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/univocity-parsers-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/azure-storage-5.4.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/netty-all-4.1.30.Final.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-container-servlet-core-2.22.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/macro-compat_2.12-1.1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-httpclient-3.1.jar":"System Classpath","/hadoop/lib/httpcore-4.4.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/hadoop/lib/java-xmlbuilder-0.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jodd-core-3.5.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/breeze-macros_2.12-0.13.2.jar":"System Classpath","/hadoop-yarn-client/lib/zookeeper-3.4.6.jar":"System Classpath","/hadoop/lib/curator-framework-2.7.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jul-to-slf4j-1.7.16.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/xz-1.5.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/antlr-2.7.7.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/json4s-scalap_2.12-3.5.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/apache-log4j-extras-1.2.17.jar":"System Classpath","/hadoop-hdfs-client/lib/htrace-core-3.1.0-incubating.jar":"System Classpath","/hadoop/hadoop-azure.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-nodemanager.jar":"System Classpath","/hadoop-yarn-client/lib/servlet-api-2.5.jar":"System Classpath","/hadoop-yarn-client/lib/guava-11.0.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/arpack_combined_all-0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/arrow-format-0.10.0.jar":"System Classpath","/hadoop-yarn-client/lib/guice-3.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/scala-library-2.12.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-container-servlet-2.22.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/xercesImpl-2.9.1.jar":"System Classpath","/hadoop/hadoop-azure-datalake-2.7.3.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-common.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/yarn/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-auth-2.7.4.jar":"System Classpath","/hadoop/lib/activation-1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jline-2.14.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/yarn/lib/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hive-metastore-1.2.1.spark2.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-applicationhistoryservice.jar":"System Classpath","/hadoop/lib/ojdbc6.jar":"System Classpath","/hadoop/lib/commons-collections-3.2.2.jar":"System Classpath","/hadoop-hdfs-client/lib/servlet-api-2.5.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/joda-time-2.9.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-annotations-2.9.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-mllib-local_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/leveldbjni-all-1.8.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-common-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/json4s-jackson_2.12-3.5.3.jar":"System Classpath","/hadoop/conf":"System Classpath","/hadoop/hadoop-common-2.7.3-tests.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-timeline-pluginstorage-2.7.3.jar":"System Classpath","/hadoop/lib/junit-4.11.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs-2.7.3-tests.jar":"System Classpath","/hadoop-yarn-client/lib/httpcore-4.4.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-unsafe_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/metrics-jvm-3.1.5.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/objenesis-2.5.1.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-nodemanager-2.7.3.jar":"System Classpath","/hadoop/hadoop-common-tests.jar":"System Classpath","/hadoop-hdfs-client/lib/jackson-databind-2.2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-graphx_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-hdfs-client/lib/asm-3.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/kryo-shaded-4.0.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/compress-lzf-1.0.3.jar":"System Classpath","/hadoop/lib/azure-storage-5.4.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/avro-mapred-1.8.2-hadoop2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/scala-xml_2.12-1.0.5.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-applications-distributedshell.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jaxb-api-2.2.2.jar":"System Classpath","/hadoop/lib/commons-net-3.1.jar":"System Classpath","/hadoop-yarn-client/lib/java-xmlbuilder-0.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/paranamer-2.8.jar":"System Classpath","/hadoop/lib/commons-logging-1.1.3.jar":"System Classpath","/hadoop-yarn-client/lib/log4j-1.2.17.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/mapreduce/*":"System Classpath","/hadoop/lib/commons-codec-1.4.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs-nfs-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/commons-lang3-3.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-math3-3.4.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-network-common_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jetty-6.1.26.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/javolution-5.5.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jcl-over-slf4j-1.7.16.jar":"System Classpath","/hadoop/lib/slf4j-api-1.7.10.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-common-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-core-asl-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hk2-utils-2.4.0-b34.jar":"System Classpath","/hadoop/lib/mockito-all-1.8.5.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-client.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/common/lib/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hppc-0.7.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/shapeless_2.12-2.3.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-mapreduce-client-common-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/log4j-1.2.17.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-module-scala_2.12-2.9.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-module-paranamer-2.9.6.jar":"System Classpath","/hadoop-hdfs-client/lib/jetty-6.1.26.hwx.jar":"System Classpath","/hadoop/hadoop-aws.jar":"System Classpath","/hadoop/lib/commons-cli-1.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-codec-1.10.jar":"System Classpath","/hadoop/lib/commons-compress-1.4.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-server-2.22.2.jar":"System Classpath","/hadoop/lib/avro-1.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-xc-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-dbcp-1.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-beanutils-1.7.0.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-jaxrs-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/api-asn1-api-1.0.0-M20.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-client-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/javax.inject-2.4.0-b34.jar":"System Classpath","/hadoop/gcs-connector-1.8.1-shaded.jar":"System Classpath","/hadoop-hdfs-client/lib/jersey-core-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/metrics-json-3.1.5.jar":"System Classpath","/hadoop/lib/jackson-core-2.2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jpam-1.1.jar":"System Classpath","/hadoop/lib/jersey-core-1.9.jar":"System Classpath","/hadoop/lib/snappy-java-1.0.4.1.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-lang-2.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/slf4j-log4j12-1.7.16.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/osgi-resource-locator-1.0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/api-asn1-api-1.0.0-M20.jar":"System Classpath","/hadoop-yarn-client/lib/nimbus-jose-jwt-4.41.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-databind-2.9.6.jar":"System Classpath","/hadoop-hdfs-client/lib/xml-apis-1.3.04.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-client-2.22.2.jar":"System Classpath","/hadoop/hadoop-nfs-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hive-jdbc-1.2.1.spark2.jar":"System Classpath","/hadoop/lib/jackson-databind-2.2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/eigenbase-properties-1.1.5.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/machinist_2.12-0.6.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/tools/lib/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-jackson-1.10.0.jar":"System Classpath","/hadoop-yarn-client/lib/jetty-util-6.1.26.hwx.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-digester-1.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/calcite-avatica-1.2.0-incubating.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/oro-2.0.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hive-cli-1.2.1.spark2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-hive-thriftserver_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-applications-unmanaged-am-launcher-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/JavaEWAH-0.3.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/super-csv-2.2.0.jar":"System Classpath","/hadoop/lib/ranger-yarn-plugin-shim-0.7.0.jar":"System Classpath","/hadoop-yarn-client/lib/jetty-6.1.26.hwx.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/metrics-core-3.1.5.jar":"System Classpath","/hadoop-yarn-client/lib/commons-compress-1.4.1.jar":"System Classpath","/hadoop-hdfs-client/lib/netty-all-4.0.52.Final.jar":"System Classpath","/hadoop/hadoop-auth.jar":"System Classpath","/hadoop-yarn-client/lib/javax.inject-1.jar":"System Classpath","/hadoop/lib/jettison-1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jetty-util-6.1.26.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-network-shuffle_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/lib/jackson-jaxrs-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/json-smart-1.3.1.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-api-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/commons-beanutils-1.7.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-streaming_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/lib/commons-io-2.4.jar":"System Classpath","/hadoop/lib/commons-beanutils-1.7.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/snappy-0.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jta-1.1.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-resourcemanager-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/protobuf-java-2.5.0.jar":"System Classpath","/hadoop-yarn-client/lib/curator-framework-2.7.1.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-daemon-1.0.13.jar":"System Classpath","/hadoop/lib/hadoop-lzo-0.6.0-sources.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-common.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/breeze_2.12-0.13.2.jar":"System Classpath","/hadoop/lib/jsp-api-2.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/orc-mapreduce-1.5.4-nohive.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-mapreduce-client-core-2.7.4.jar":"System Classpath","/hadoop/lib/htrace-core-3.1.0-incubating.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-encoding-1.10.0.jar":"System Classpath","/hadoop/hadoop-auth-2.7.3.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-applicationhistoryservice-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/jettison-1.1.jar":"System Classpath","/hadoop-yarn-client/lib/commons-codec-1.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-mapreduce-client-shuffle-2.7.4.jar":"System Classpath","/hadoop/lib/curator-recipes-2.7.1.jar":"System Classpath","/hadoop/lib/jetty-util-6.1.26.hwx.jar":"System Classpath","/hadoop-hdfs-client/lib/log4j-1.2.17.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/activation-1.1.1.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/jaxb-api-2.2.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/opencsv-2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-collections-3.2.2.jar":"System Classpath","/hadoop/lib/guava-11.0.2.jar":"System Classpath","/hadoop-yarn-client/lib/commons-net-3.1.jar":"System Classpath","/hadoop/lib/commons-beanutils-core-1.8.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/py4j-0.10.8.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/json4s-ast_2.12-3.5.3.jar":"System Classpath","/hadoop-yarn-client/lib/metrics-core-3.0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/pyrolite-4.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/chill_2.12-0.9.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/libfb303-0.9.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/javax.inject-1.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs.jar":"System Classpath","/ext/hadoop/*":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-web-proxy-2.7.3.jar":"System Classpath","/hadoop/lib/aws-java-sdk-core-1.10.6.jar":"System Classpath","/hadoop-yarn-client/lib/commons-configuration-1.6.jar":"System Classpath","/hadoop/lib/zookeeper-3.4.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-logging-1.1.3.jar":"System Classpath","/hadoop-hdfs-client/lib/jsr305-3.0.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/stringtemplate-3.2.1.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-resourcemanager.jar":"System Classpath","/hadoop/lib/netty-3.6.2.Final.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/zstd-jni-1.3.2-2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jsp-api-2.1.jar":"System Classpath","/hadoop-yarn-client/lib/jaxb-impl-2.2.3-1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-tags_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hive-beeline-1.2.1.spark2.jar":"System Classpath","/hadoop-hdfs-client/lib/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/snappy-java-1.0.4.1.jar":"System Classpath","/hadoop-yarn-client/lib/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs-nfs.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/hadoop/hadoop-aws-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/jsr305-3.0.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/datanucleus-rdbms-3.2.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/libthrift-0.9.3.jar":"System Classpath","/hadoop/lib/xz-1.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/json4s-core_2.12-3.5.3.jar":"System Classpath","/hadoop-yarn-client/lib/jersey-guice-1.9.jar":"System Classpath","/hadoop-yarn-client/lib/activation-1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-jaxrs-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/hdfs/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/mapreduce/lib/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-pool-1.5.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/arrow-memory-0.10.0.jar":"System Classpath","/hadoop-hdfs-client/lib/jetty-util-6.1.26.hwx.jar":"System Classpath","/hadoop/lib/ranger-plugin-classloader-0.7.0.jar":"System Classpath","/hadoop/lib/aws-java-sdk-kms-1.10.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-repl_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-yarn-client/lib/jersey-json-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-yarn_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-yarn-client/lib/xz-1.0.jar":"System Classpath","/hadoop/hadoop-common-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jetty-sslengine-6.1.26.jar":"System Classpath","/hadoop/lib/jaxb-api-2.2.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/metrics-graphite-3.1.5.jar":"System Classpath","/hadoop/lib/jcip-annotations-1.0-1.jar":"System Classpath","/hadoop/lib/nimbus-jose-jwt-4.41.1.jar":"System Classpath","/hadoop/lib/hadoop-lzo-0.6.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hk2-locator-2.4.0-b34.jar":"System Classpath","/hadoop/lib/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-logging-1.1.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jsr305-3.0.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-hive_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/RoaringBitmap-0.5.11.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/calcite-linq4j-1.2.0-incubating.jar":"System Classpath","/hadoop-yarn-client/lib/commons-beanutils-core-1.8.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-lang-2.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/derby-10.12.1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-hadoop-1.10.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/stax-api-1.0-2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/scala-reflect-2.12.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/calcite-core-1.2.0-incubating.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_conf__":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-format-2.4.0.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-sharedcachemanager-2.7.3.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-web-proxy.jar":"System Classpath","/hadoop-hdfs-client/lib/xercesImpl-2.9.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/netty-3.9.9.Final.jar":"System Classpath","/hadoop/lib/xmlenc-0.52.jar":"System Classpath","/hadoop/hadoop-annotations-2.7.3.jar":"System Classpath","/hadoop-hdfs-client/lib/protobuf-java-2.5.0.jar":"System Classpath","/hadoop-yarn-client/lib/jersey-core-1.9.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-registry-2.7.3.jar":"System Classpath","/hadoop-hdfs-client/lib/okhttp-2.7.5.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-yarn-common-2.7.4.jar":"System Classpath","/hadoop-yarn-client/lib/leveldbjni-all-1.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-crypto-1.0.0.jar":"System Classpath","/hadoop/lib/jetty-6.1.26.hwx.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/aopalliance-repackaged-2.4.0-b34.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_conf__/__hadoop_conf__":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-compress-1.8.1.jar":"System Classpath","/hadoop/lib/jackson-xc-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/gson-2.2.4.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-api.jar":"System Classpath","/hadoop/lib/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs-tests.jar":"System Classpath","/hadoop-hdfs-client/lib/commons-io-2.4.jar":"System Classpath","/hadoop/hadoop-nfs.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-yarn-api-2.7.4.jar":"System Classpath","/hadoop/lib/json-smart-1.3.1.jar":"System Classpath","/hadoop-yarn-client/lib/commons-lang-2.6.jar":"System Classpath","/hadoop-hdfs-client/hadoop-hdfs-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/azure-keyvault-core-0.8.0.jar":"System Classpath","/hadoop-yarn-client/lib/fst-2.24.jar":"System Classpath","/hadoop-yarn-client/lib/htrace-core-3.1.0-incubating.jar":"System Classpath","/hadoop/hadoop-azure-datalake.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-common-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hive-exec-1.2.1.spark2.jar":"System Classpath","/hadoop-hdfs-client/lib/jersey-server-1.9.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-sharedcachemanager.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-launcher_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/datanucleus-core-3.2.10.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/zookeeper-3.4.6.jar":"System Classpath","/hadoop-yarn-client/lib/commons-math3-3.1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/api-util-1.0.0-M20.jar":"System Classpath","/hadoop/lib/azure-keyvault-core-0.8.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-common-2.22.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jtransforms-2.4.0.jar":"System Classpath","/hadoop/lib/asm-3.2.jar":"System Classpath","/hadoop/lib/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/javax.ws.rs-api-2.0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/xmlenc-0.52.jar":"System Classpath","/hadoop/lib/jsch-0.1.54.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-timeline-pluginstorage.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-annotations-2.2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/hdfs/lib/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/httpclient-4.5.6.jar":"System Classpath","/hadoop-hdfs-client/lib/xmlenc-0.52.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-annotations-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/antlr-runtime-3.4.jar":"System Classpath","/etc/hadoop/conf/secure":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/avro-ipc-1.8.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/janino-3.0.11.jar":"System Classpath","/hadoop/lib/api-util-1.0.0-M20.jar":"System Classpath","/hadoop-yarn-client/lib/xmlenc-0.52.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/slf4j-api-1.7.16.jar":"System Classpath","/hadoop/lib/ranger-hdfs-plugin-shim-0.7.0.jar":"System Classpath","/hadoop-hdfs-client/lib/guava-11.0.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/chill-java-0.9.3.jar":"System Classpath","/hadoop-yarn-client/lib/aopalliance-1.0.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-applications-unmanaged-am-launcher.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/htrace-core-3.1.0-incubating.jar":"System Classpath","/hadoop/lib/jersey-json-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/guava-14.0.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-yarn-server-web-proxy-2.7.4.jar":"System Classpath","/hadoop/lib/jetty-sslengine-6.1.26.hwx.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/ST4-4.0.4.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-client-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-sketch_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-yarn-client/lib/httpclient-4.5.2.jar":"System Classpath","/hadoop/lib/httpclient-4.5.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/antlr4-runtime-4.7.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/arrow-vector-0.10.0.jar":"System Classpath","/hadoop-yarn-client/lib/jersey-server-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-yarn-client-2.7.4.jar":"System Classpath","/hadoop-yarn-client/lib/jsch-0.1.54.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/curator-client-2.7.1.jar":"System Classpath","/hadoop-yarn-client/lib/stax-api-1.0-2.jar":"System Classpath","/hadoop-yarn-client/lib/commons-cli-1.2.jar":"System Classpath","/hadoop/lib/servlet-api-2.5.jar":"System Classpath","/hadoop-yarn-client/lib/commons-collections-3.2.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-net-3.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jersey-media-jaxb-2.22.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-mapreduce-client-app-2.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-core_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-catalyst_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-kvstore_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/lib/joda-time-2.9.4.jar":"System Classpath","/hadoop-yarn-client/lib/jersey-client-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-lang3-3.8.1.jar":"System Classpath","/hadoop/lib/slf4j-log4j12-1.7.10.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/bonecp-0.8.0.RELEASE.jar":"System Classpath","/hadoop-yarn-client/lib/jetty-sslengine-6.1.26.hwx.jar":"System Classpath","/hadoop-yarn-client/lib/commons-logging-1.1.3.jar":"System Classpath","/hadoop/lib/jets3t-0.9.0.jar":"System Classpath","/hadoop/lib/hadoop-lzo-0.6.0-javadoc.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/orc-shims-1.5.4.jar":"System Classpath","/hadoop-hdfs-client/lib/okio-1.6.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-column-1.10.0.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-core-asl-1.9.13.jar":"System Classpath","/hadoop-yarn-client/lib/jsp-api-2.1.jar":"System Classpath","/hadoop/lib/aws-java-sdk-s3-1.10.6.jar":"System Classpath","/hadoop/azure-data-lake-store-sdk-2.2.5.jar":"System Classpath","/hadoop-yarn-client/lib/zookeeper-3.4.6-tests.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/stream-2.7.0.jar":"System Classpath","/hadoop/hadoop-azure-2.7.3.jar":"System Classpath","/hadoop-yarn-client/lib/jcip-annotations-1.0-1.jar":"System Classpath","/hadoop-yarn-client/lib/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/mr-framework/hadoop/share/hadoop/common/*":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/aircompressor-0.10.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/httpcore-4.4.10.jar":"System Classpath","/hadoop-yarn-client/lib/objenesis-2.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/avro-1.8.2.jar":"System Classpath","/hadoop-yarn-client/lib/jets3t-0.9.0.jar":"System Classpath","/hadoop-hdfs-client/lib/jackson-core-2.2.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-configuration-1.6.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-mapreduce-client-jobclient-2.7.4.jar":"System Classpath","/hadoop-yarn-client/lib/commons-io-2.4.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-tests.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/guice-3.0.jar":"System Classpath","/hadoop/lib/jsr305-3.0.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/javassist-3.18.1-GA.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/curator-framework-2.7.1.jar":"System Classpath","/hadoop-yarn-client/lib/avro-1.7.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/scala-compiler-2.12.8.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-registry.jar":"System Classpath","/hadoop/lib/stax-api-1.0-2.jar":"System Classpath","/hadoop-yarn-client/lib/guice-servlet-3.0.jar":"System Classpath","/hadoop/lib/commons-math3-3.1.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/javax.annotation-api-1.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jdo-api-3.0.1.jar":"System Classpath","/hadoop-hdfs-client/lib/jackson-core-asl-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-sql_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/hadoop-yarn-server-common-2.7.4.jar":"System Classpath","/hadoop-yarn-client/lib/api-util-1.0.0-M20.jar":"System Classpath","/hadoop/lib/api-asn1-api-1.0.0-M20.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/minlog-1.3.0.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-xc-1.9.13.jar":"System Classpath","/hadoop-hdfs-client/lib/netty-3.6.2.Final.jar":"System Classpath","/hadoop/lib/hamcrest-core-1.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/xbean-asm7-shaded-4.12.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-core-2.2.3.jar":"System Classpath","/hadoop/lib/jaxb-impl-2.2.3-1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/scala-parser-combinators_2.12-1.1.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-io-2.4.jar":"System Classpath","/hadoop/lib/commons-lang-2.6.jar":"System Classpath","/hadoop/lib/commons-lang3-3.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spire_2.12-0.13.0.jar":"System Classpath","/hadoop-hdfs-client/lib/jackson-annotations-2.2.3.jar":"System Classpath","/hadoop/lib/jackson-core-asl-1.9.13.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/javax.servlet-api-3.1.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/core-1.1.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-cli-1.2.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/curator-recipes-2.7.1.jar":"System Classpath","/hadoop/lib/jersey-server-1.9.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spire-macros_2.12-0.13.0.jar":"System Classpath","/hadoop-yarn-client/lib/javassist-3.18.1-GA.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/validation-api-1.1.0.Final.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/protobuf-java-2.5.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/commons-beanutils-core-1.8.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/gson-2.2.4.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/ivy-2.4.0.jar":"System Classpath","/hadoop-yarn-client/lib/paranamer-2.3.jar":"System Classpath","/hadoop/lib/gson-2.2.4.jar":"System Classpath","/hadoop/lib/jackson-annotations-2.2.3.jar":"System Classpath","/hadoop-yarn-client/lib/jackson-databind-2.2.3.jar":"System Classpath","/hadoop-hdfs-client/lib/leveldbjni-all-1.8.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/guice-servlet-3.0.jar":"System Classpath","/hadoop/hadoop-annotations.jar":"System Classpath","/hadoop-yarn-client/lib/curator-recipes-2.7.1.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/flatbuffers-1.2.0-3f79e055.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/spark-mllib_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop-yarn-client/hadoop-yarn-server-tests-2.7.3.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/parquet-common-1.10.0.jar":"System Classpath","/hadoop-yarn-client/lib/asm-3.2.jar":"System Classpath","/hadoop/lib/protobuf-java-2.5.0.jar":"System Classpath","/hadoop/yarn/local/usercache/spark/appcache/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001/__spark_libs__/jackson-core-2.9.6.jar":"System Classpath","/hadoop/lib/commons-configuration-1.6.jar":"System Classpath"}} -{"Event":"SparkListenerApplicationStart","App Name":"Spark SQL basic example","App ID":"application_1547723113049_0006","Timestamp":1547788454200,"User":"spark","App Attempt ID":"1","Driver Logs":{"stdout":"http://node2:8042/node/containerlogs/container_e01_1547723113049_0005_01_000001/spark/stdout?start=-4096","stderr":"http://node2:8042/node/containerlogs/container_e01_1547723113049_0005_01_000001/spark/stderr?start=-4096"},"Driver Attributes":{"NODE_HTTP_ADDRESS":"node2:8042","USER":"spark","LOG_FILES":"stderr,stdout","CLUSTER_ID":"yarn-cluster","HTTP_SCHEME":"http://","CONTAINER_ID":"container_e01_1547723113049_0005_01_000001"}} -{"Event":"SparkListenerExecutorAdded","Timestamp":1547788454209,"Executor ID":"1","Executor Info":{"Host":"node3","Total Cores":1,"Log Urls":{"stdout":"http://node3:8042/node/containerlogs/container_e01_1547723113049_0005_01_000002/spark/stdout?start=-4096","stderr":"http://node3:8042/node/containerlogs/container_e01_1547723113049_0005_01_000002/spark/stderr?start=-4096"},"Attributes":{"NODE_HTTP_ADDRESS":"node3:8042","USER":"spark","LOG_FILES":"stderr,stdout","CLUSTER_ID":"yarn-cluster","HTTP_SCHEME":"http://","CONTAINER_ID":"container_e01_1547723113049_0005_01_000002"}}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"node3","Port":35087},"Maximum Memory":384093388,"Timestamp":1547788454413,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} -{"Event":"SparkListenerExecutorAdded","Timestamp":1547788455343,"Executor ID":"2","Executor Info":{"Host":"node4","Total Cores":1,"Log Urls":{"stdout":"http://node4:8042/node/containerlogs/container_e01_1547723113049_0005_01_000003/spark/stdout?start=-4096","stderr":"http://node4:8042/node/containerlogs/container_e01_1547723113049_0005_01_000003/spark/stderr?start=-4096"},"Attributes":{"NODE_HTTP_ADDRESS":"node4:8042","USER":"spark","LOG_FILES":"stderr,stdout","CLUSTER_ID":"yarn-cluster","HTTP_SCHEME":"http://","CONTAINER_ID":"container_e01_1547723113049_0005_01_000003"}}} -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"2","Host":"node4","Port":44875},"Maximum Memory":384093388,"Timestamp":1547788455624,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} -{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1547788459293,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"json at SparkSQLExample.scala:59","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"mapPartitions\"}","Callsite":"json at SparkSQLExample.scala:59","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"json at SparkSQLExample.scala:59","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"json at SparkSQLExample.scala:59","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameReader.json(DataFrameReader.scala:331)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:59)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Accumulables":[]}],"Stage IDs":[0],"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0006","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0006","spark.app.id":"application_1547723113049_0006","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"json at SparkSQLExample.scala:59","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"mapPartitions\"}","Callsite":"json at SparkSQLExample.scala:59","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"json at SparkSQLExample.scala:59","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"json at SparkSQLExample.scala:59","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":1,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.DataFrameReader.json(DataFrameReader.scala:331)\norg.apache.spark.examples.sql.SparkSQLExample$.runBasicDataFrameExample(SparkSQLExample.scala:59)\norg.apache.spark.examples.sql.SparkSQLExample$.main(SparkSQLExample.scala:49)\norg.apache.spark.examples.sql.SparkSQLExample.main(SparkSQLExample.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:659)","Submission Time":1547788459321,"Accumulables":[]},"Properties":{"spark.history.kerberos.keytab":"none","spark.driver.host":"node2","spark.history.fs.logDirectory":"hdfs:///spark2-history/","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"35589","spark.yarn.queue":"default","spark.some.config.option":"some-value","spark.yarn.historyServer.address":"node5:18081","spark.yarn.app.id":"application_1547723113049_0006","spark.history.custom.executor.log.url":"{{HTTP_SCHEME}}host:port/testurl/node_http_address/{{NODE_HTTP_ADDRESS}}/cluster_id/{{CLUSTER_ID}}/container_id/{{CONTAINER_ID}}/user/{{USER}}/file/{{FILE_NAME}}","spark.app.name":"Spark SQL basic example","spark.history.kerberos.principal":"none","spark.submit.pyFiles":"","spark.yarn.am.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.executor.id":"driver","spark.yarn.app.container.log.dir":"/hadoop/yarn/log/application_1547723113049_0006/container_e01_1547723113049_0005_01_000001","spark.driver.extraJavaOptions":"-Dhdp.version=2.6.5.1002-73","spark.submit.deployMode":"cluster","spark.master":"yarn","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.history.provider":"org.apache.spark.deploy.history.FsHistoryProvider","spark.eventLog.dir":"hdfs:///spark2-history/","spark.history.ui.port":"18081","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"node6","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://node6:8088/proxy/application_1547723113049_0006","spark.app.id":"application_1547723113049_0006","spark.hadoop.yarn.timeline-service.enabled":"false"}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1547788459627,"Executor ID":"1","Host":"node3","Locality":"NODE_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":1547788459627,"Executor ID":"1","Host":"node3","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1547788463652,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"duration total (min, med, max)","Update":"1582","Value":"1581","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":0,"Name":"number of output rows","Update":"3","Value":"3","Internal":true,"Count Failed Values":true,"Metadata":"sql"},{"ID":27,"Name":"internal.metrics.input.recordsRead","Update":3,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":26,"Name":"internal.metrics.input.bytesRead","Update":73,"Value":73,"Internal":true,"Count Failed Values":true},{"ID":11,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":10,"Name":"internal.metrics.jvmGCTime","Update":159,"Value":159,"Internal":true,"Count Failed Values":true},{"ID":9,"Name":"internal.metrics.resultSize","Update":1963,"Value":1963,"Internal":true,"Count Failed Values":true},{"ID":8,"Name":"internal.metrics.executorCpuTime","Update":2225128499,"Value":2225128499,"Internal":true,"Count Failed Values":true},{"ID":7,"Name":"internal.metrics.executorRunTime","Update":2291,"Value":2291,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1171850277,"Value":1171850277,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.executorDeserializeTime","Update":1431,"Value":1431,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1431,"Executor Deserialize CPU Time":1171850277,"Executor Run Time":2291,"Executor CPU Time":2225128499,"Result Size":1963,"JVM GC Time":159,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":73,"Records Read":3},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 7d0e67fd40f40..7cf0703868893 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -294,10 +294,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc test("log urls without customization") { val conf = createTestConf() - val appId = "app1" - val user = "user1" - - val executorInfos = (1 to 5).map(createTestExecutorInfo(appId, user, _)) + val executorInfos = (1 to 5).map(createTestExecutorInfo("app1", "user1", _)) val expected: Map[ExecutorInfo, Map[String, String]] = executorInfos.map { execInfo => execInfo -> execInfo.logUrlMap @@ -313,10 +310,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc // some of available attributes are not used in pattern which should be OK - val appId = "app1" - val user = "user1" - - val executorInfos = (1 to 5).map(createTestExecutorInfo(appId, user, _)) + val executorInfos = (1 to 5).map(createTestExecutorInfo("app1", "user1", _)) val expected: Map[ExecutorInfo, Map[String, String]] = executorInfos.map { execInfo => val attr = execInfo.attributes @@ -339,10 +333,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc // some of available attributes are not used in pattern which should be OK - val appId = "app1" - val user = "user1" - - val executorInfos = (1 to 5).map(createTestExecutorInfo(appId, user, _)) + val executorInfos = (1 to 5).map(createTestExecutorInfo("app1", "user1", _)) val expected: Map[ExecutorInfo, Map[String, String]] = executorInfos.map { execInfo => val attr = execInfo.attributes @@ -363,13 +354,9 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc .set(CUSTOM_EXECUTOR_LOG_URL, "http://newhost:9999/logs/clusters/{{CLUSTER_ID}}" + "/users/{{USER}}/containers/{{CONTAINER_ID}}/{{NON_EXISTING}}/{{FILE_NAME}}") - val appId = "app1" - val user = "user1" - - val executorInfos = (1 to 5).map(createTestExecutorInfo(appId, user, _)) + val executorInfos = (1 to 5).map(createTestExecutorInfo("app1", "user1", _)) val expected: Map[ExecutorInfo, Map[String, String]] = executorInfos.map { execInfo => - execInfo -> execInfo.logUrlMap }.toMap @@ -383,11 +370,8 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc .set(CUSTOM_EXECUTOR_LOG_URL, "http://newhost:9999/logs/clusters/{{CLUSTER_ID}}" + "/users/{{USER}}/containers/{{CONTAINER_ID}}/{{NON_EXISTING}}/{{FILE_NAME}}") - val appId = "app1" - val user = "user1" - val executorInfos = (1 to 5).map( - createTestExecutorInfo(appId, user, _, includingLogFiles = false)) + createTestExecutorInfo("app1", "user1", _, includingLogFiles = false)) val expected: Map[ExecutorInfo, Map[String, String]] = executorInfos.map { execInfo => execInfo -> execInfo.logUrlMap @@ -396,19 +380,61 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc testHandlingExecutorLogUrl(conf, expected) } + test("custom log urls, app not finished, applyIncompleteApplication: true") { + val conf = createTestConf() + .set(CUSTOM_EXECUTOR_LOG_URL, "http://newhost:9999/logs/clusters/{{CLUSTER_ID}}" + + "/users/{{USER}}/containers/{{CONTAINER_ID}}/{{FILE_NAME}}") + .set(APPLY_CUSTOM_EXECUTOR_LOG_URL_TO_INCOMPLETE_APP, true) + + // ensure custom log urls are applied to incomplete application + + val executorInfos = (1 to 5).map(createTestExecutorInfo("app1", "user1", _)) + + val expected: Map[ExecutorInfo, Map[String, String]] = executorInfos.map { execInfo => + val attr = execInfo.attributes + val newLogUrlMap = attr("LOG_FILES").split(",").map { file => + val newLogUrl = s"http://newhost:9999/logs/clusters/${attr("CLUSTER_ID")}" + + s"/users/${attr("USER")}/containers/${attr("CONTAINER_ID")}/$file" + file -> newLogUrl + }.toMap + + execInfo -> newLogUrlMap + }.toMap + + testHandlingExecutorLogUrl(conf, expected, isCompletedApp = false) + } + + test("custom log urls, app not finished, applyIncompleteApplication: false") { + val conf = createTestConf() + .set(CUSTOM_EXECUTOR_LOG_URL, "http://newhost:9999/logs/clusters/{{CLUSTER_ID}}" + + "/users/{{USER}}/containers/{{CONTAINER_ID}}/{{FILE_NAME}}") + .set(APPLY_CUSTOM_EXECUTOR_LOG_URL_TO_INCOMPLETE_APP, false) + + // ensure custom log urls are NOT applied to incomplete application + + val executorInfos = (1 to 5).map(createTestExecutorInfo("app1", "user1", _)) + + val expected: Map[ExecutorInfo, Map[String, String]] = executorInfos.map { execInfo => + execInfo -> execInfo.logUrlMap + }.toMap + + testHandlingExecutorLogUrl(conf, expected, isCompletedApp = false) + } + private def testHandlingExecutorLogUrl( conf: SparkConf, - expectedLogUrlMap: Map[ExecutorInfo, Map[String, String]]): Unit = { + expectedLogUrlMap: Map[ExecutorInfo, Map[String, String]], + isCompletedApp: Boolean = true): Unit = { val provider = new FsHistoryProvider(conf) val attempt1 = newLogFile("app1", Some("attempt1"), inProgress = true) val executorAddedEvents = expectedLogUrlMap.keys.zipWithIndex.map { case (execInfo, idx) => - val event = SparkListenerExecutorAdded(1 + idx, s"exec$idx", execInfo) - event + SparkListenerExecutorAdded(1 + idx, s"exec$idx", execInfo) }.toList.sortBy(_.time) val allEvents = List(SparkListenerApplicationStart("app1", Some("app1"), 1L, - "test", Some("attempt1"))) ++ executorAddedEvents + "test", Some("attempt1"))) ++ executorAddedEvents ++ + (if (isCompletedApp) List(SparkListenerApplicationEnd(1000L)) else Seq()) writeFile(attempt1, true, None, allEvents: _*) 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 fb358a1290b22..1a071fa771330 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 @@ -109,8 +109,6 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers } } - override protected def afterAll(): Unit = stop() - val cases = Seq( "application list json" -> "applications", "completed app list json" -> "applications?status=completed", @@ -185,42 +183,19 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers // in the test resource folder cases.foreach { case (name, path) => test(name) { - val expectationFile = new File(expRoot, HistoryServerSuite.sanitizePath(name) + - "_expectation.json") - assertApiCallResponse(path, expectationFile) - } - } - - val casesCustomLogUrl = Seq( - "completed app executor list json apply custom log urls" -> - "applications/application_1547723113049_0005/1/executors", - "incomplete app executor list json apply custom log urls" -> - "applications/application_1547723113049_0006/1/executors" - ) - - casesCustomLogUrl.foreach { case (name, path) => - Seq(true, false).foreach { applyToIncompleteApp => - val newName = name + s" apply incomplete app $applyToIncompleteApp" - test(newName) { - // This only verifies whether applying custom log URLs is in effect for SHS. - // Validation of "custom log URLs" functionality will be covered from different UTs. - - // restart server - try { - stop() - init( - CUSTOM_EXECUTOR_LOG_URL.key -> ("http://newhost:9999/logs/clusters/" + - "{{CLUSTER_ID}}/users/{{USER}}/containers/{{CONTAINER_ID}}/{{FILE_NAME}}"), - APPLY_CUSTOM_EXECUTOR_LOG_URL_TO_INCOMPLETE_APP.key -> applyToIncompleteApp.toString) - - val expectationFile = new File(expRoot, HistoryServerSuite.sanitizePath(newName) + - "_expectation.json") - assertApiCallResponse(path, expectationFile) - } finally { - // make sure other UTs are not affected from relaunching HistoryServer - stop() - } - } + val (code, jsonOpt, errOpt) = getContentAndCode(path) + code should be (HttpServletResponse.SC_OK) + jsonOpt should be ('defined) + errOpt should be (None) + + val exp = IOUtils.toString(new FileInputStream( + new File(expRoot, HistoryServerSuite.sanitizePath(name) + "_expectation.json"))) + // compare the ASTs so formatting differences don't cause failures + import org.json4s._ + import org.json4s.jackson.JsonMethods._ + val jsonAst = parse(clearLastUpdated(jsonOpt.get)) + val expAst = parse(exp) + assertValidDataInJson(jsonAst, expAst) } } @@ -688,21 +663,6 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers out.close() } - private def assertApiCallResponse(path: String, expectationFile: File): Unit = { - val (code, jsonOpt, errOpt) = getContentAndCode(path) - code should be (HttpServletResponse.SC_OK) - jsonOpt should be ('defined) - errOpt should be (None) - - val exp = IOUtils.toString(new FileInputStream(expectationFile)) - // compare the ASTs so formatting differences don't cause failures - import org.json4s._ - import org.json4s.jackson.JsonMethods._ - val jsonAst = parse(clearLastUpdated(jsonOpt.get)) - val expAst = parse(exp) - assertValidDataInJson(jsonAst, expAst) - } - } object HistoryServerSuite { diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 8995c83cbbfa5..f9807b63540ee 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -39,6 +39,12 @@ object MimaExcludes { // [SPARK-26254][CORE] Extract Hive + Kafka dependencies from Core. ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.deploy.security.HiveDelegationTokenProvider"), + // [SPARK-26311][CORE]New feature: apply custom log URL pattern for executor log URLs + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerApplicationStart.apply"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerApplicationStart.copy"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerApplicationStart.this"), + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.scheduler.SparkListenerApplicationStart$"), + // [SPARK-25765][ML] Add training cost to BisectingKMeans summary ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.clustering.BisectingKMeansModel.this"), @@ -280,13 +286,7 @@ object MimaExcludes { // [SPARK-26616][MLlib] Expose document frequency in IDFModel ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.mllib.feature.IDFModel.this"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.mllib.feature.IDF#DocumentFrequencyAggregator.idf"), - - // [SPARK-26311][CORE]New feature: apply custom log URL pattern for executor log URLs - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerApplicationStart.apply"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerApplicationStart.copy"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerApplicationStart.this"), - ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.scheduler.SparkListenerApplicationStart$") + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.mllib.feature.IDF#DocumentFrequencyAggregator.idf") ) // Exclude rules for 2.4.x diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 74c30f5211e1f..ce5c097c334f1 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.yarn import java.io.File import java.nio.ByteBuffer -import java.util.Collections +import java.util.{Collections, Locale} import scala.collection.JavaConverters._ import scala.collection.mutable.{HashMap, ListBuffer} @@ -34,13 +34,13 @@ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.client.api.NMClient import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC -import org.apache.hadoop.yarn.util.{ConverterUtils, Records} +import org.apache.hadoop.yarn.util.Records import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.network.util.JavaUtils -import org.apache.spark.util.Utils +import org.apache.spark.util.{Utils, YarnExecutorHelper} private[yarn] class ExecutorRunnable( container: Option[Container], @@ -227,13 +227,6 @@ private[yarn] class ExecutorRunnable( val env = new HashMap[String, String]() Client.populateClasspath(null, conf, sparkConf, env, sparkConf.get(EXECUTOR_CLASS_PATH)) - // lookup appropriate http scheme for container log urls - val yarnHttpPolicy = conf.get( - YarnConfiguration.YARN_HTTP_POLICY_KEY, - YarnConfiguration.YARN_HTTP_POLICY_DEFAULT - ) - val httpScheme = if (yarnHttpPolicy == "HTTPS_ONLY") "https://" else "http://" - System.getenv().asScala.filterKeys(_.startsWith("SPARK")) .foreach { case (k, v) => env(k) = v } @@ -250,22 +243,16 @@ private[yarn] class ExecutorRunnable( // Add log urls, as well as executor attributes container.foreach { c => - val containerId = ConverterUtils.toString(c.getId) - val address = c.getNodeHttpAddress - - env("SPARK_EXECUTOR_ATTRIBUTE_HTTP_SCHEME") = httpScheme - env("SPARK_EXECUTOR_ATTRIBUTE_NODE_HTTP_ADDRESS") = address - env("SPARK_EXECUTOR_ATTRIBUTE_CLUSTER_ID") = clusterId.getOrElse("") - env("SPARK_EXECUTOR_ATTRIBUTE_CONTAINER_ID") = containerId - - sys.env.filterKeys(_.endsWith("USER")).foreach { u => - val user = u._2 - val baseUrl = s"$httpScheme$address/node/containerlogs/$containerId/$user" - env("SPARK_LOG_URL_STDERR") = s"$baseUrl/stderr?start=-4096" - env("SPARK_LOG_URL_STDOUT") = s"$baseUrl/stdout?start=-4096" - - env("SPARK_EXECUTOR_ATTRIBUTE_USER") = user - env("SPARK_EXECUTOR_ATTRIBUTE_LOG_FILES") = "stderr,stdout" + YarnExecutorHelper.getLogUrls(conf, Some(c)).foreach { m => + m.foreach { case (fileName, url) => + env("SPARK_LOG_URL_" + fileName.toUpperCase(Locale.ROOT)) = url + } + } + + YarnExecutorHelper.getAttributes(conf, Some(c)).foreach { m => + m.foreach { case (attr, value) => + env("SPARK_EXECUTOR_ATTRIBUTE_" + attr.toUpperCase(Locale.ROOT)) = value + } } } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala index 42d266bfff91e..58acdeb0ed64e 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala @@ -17,16 +17,10 @@ package org.apache.spark.scheduler.cluster -import org.apache.hadoop.HadoopIllegalArgumentException -import org.apache.hadoop.yarn.api.ApplicationConstants.Environment -import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.util.ConverterUtils - import org.apache.spark.SparkContext -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.deploy.yarn.{ApplicationMaster, YarnSparkHadoopUtil} +import org.apache.spark.deploy.yarn.ApplicationMaster import org.apache.spark.scheduler.TaskSchedulerImpl -import org.apache.spark.util.Utils +import org.apache.spark.util.YarnExecutorHelper private[spark] class YarnClusterSchedulerBackend( scheduler: TaskSchedulerImpl, @@ -42,67 +36,10 @@ private[spark] class YarnClusterSchedulerBackend( } override def getDriverLogUrls: Option[Map[String, String]] = { - var driverLogs: Option[Map[String, String]] = None - try { - val yarnConf = new YarnConfiguration(sc.hadoopConfiguration) - val containerId = YarnSparkHadoopUtil.getContainerId - - val httpAddress = System.getenv(Environment.NM_HOST.name()) + - ":" + System.getenv(Environment.NM_HTTP_PORT.name()) - // lookup appropriate http scheme for container log urls - val yarnHttpPolicy = yarnConf.get( - YarnConfiguration.YARN_HTTP_POLICY_KEY, - YarnConfiguration.YARN_HTTP_POLICY_DEFAULT - ) - val user = Utils.getCurrentUserName() - val httpScheme = if (yarnHttpPolicy == "HTTPS_ONLY") "https://" else "http://" - val baseUrl = s"$httpScheme$httpAddress/node/containerlogs/$containerId/$user" - logDebug(s"Base URL for logs: $baseUrl") - driverLogs = Some(Map( - "stdout" -> s"$baseUrl/stdout?start=-4096", - "stderr" -> s"$baseUrl/stderr?start=-4096")) - } catch { - case e: Exception => - logInfo("Error while building AM log links, so AM" + - " logs link will not appear in application UI", e) - } - driverLogs + YarnExecutorHelper.getLogUrls(sc.hadoopConfiguration, container = None) } override def getDriverAttributes: Option[Map[String, String]] = { - try { - val yarnConf = new YarnConfiguration(sc.hadoopConfiguration) - val containerId = YarnSparkHadoopUtil.getContainerId - val clusterId: Option[String] = try { - Some(YarnConfiguration.getClusterId(yarnConf)) - } catch { - case _: HadoopIllegalArgumentException => None - } - - val httpAddress = System.getenv(Environment.NM_HOST.name()) + - ":" + System.getenv(Environment.NM_HTTP_PORT.name()) - - // lookup appropriate http scheme for container log urls - val yarnHttpPolicy = yarnConf.get( - YarnConfiguration.YARN_HTTP_POLICY_KEY, - YarnConfiguration.YARN_HTTP_POLICY_DEFAULT - ) - val user = Utils.getCurrentUserName() - val httpScheme = if (yarnHttpPolicy == "HTTPS_ONLY") "https://" else "http://" - - Some(Map( - "HTTP_SCHEME" -> httpScheme, - "NODE_HTTP_ADDRESS" -> httpAddress, - "CLUSTER_ID" -> clusterId.getOrElse(""), - "CONTAINER_ID" -> ConverterUtils.toString(containerId), - "USER" -> user, - "LOG_FILES" -> "stderr,stdout" - )) - } catch { - case e: Exception => - logInfo("Error while retrieving attributes on driver, so driver logs will not " + - "be replaced with custom log pattern", e) - None - } + YarnExecutorHelper.getAttributes(sc.hadoopConfiguration, container = None) } } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/util/YarnExecutorHelper.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/util/YarnExecutorHelper.scala new file mode 100644 index 0000000000000..992b4102e98b9 --- /dev/null +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/util/YarnExecutorHelper.scala @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import org.apache.hadoop.HadoopIllegalArgumentException +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.yarn.api.ApplicationConstants.Environment +import org.apache.hadoop.yarn.api.records.{Container, ContainerId} +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.util.ConverterUtils + +import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil +import org.apache.spark.internal.Logging + +private[spark] object YarnExecutorHelper extends Logging { + def getLogUrls( + conf: Configuration, + container: Option[Container]): Option[Map[String, String]] = { + try { + val yarnConf = new YarnConfiguration(conf) + + val containerId = getContainerId(container) + val httpAddress = getNodeManagerHttpAddress + val user = Utils.getCurrentUserName() + val httpScheme = getYarnHttpScheme(yarnConf) + + val baseUrl = s"$httpScheme$httpAddress/node/containerlogs/$containerId/$user" + logDebug(s"Base URL for logs: $baseUrl") + + Some(Map( + "stdout" -> s"$baseUrl/stdout?start=-4096", + "stderr" -> s"$baseUrl/stderr?start=-4096")) + } catch { + case e: Exception => + logInfo("Error while building executor logs - executor logs will not be available", e) + None + } + } + + def getAttributes( + conf: Configuration, + container: Option[Container]): Option[Map[String, String]] = { + try { + val yarnConf = new YarnConfiguration(conf) + + val clusterId = getClusterId(yarnConf) + val containerId = getContainerId(container) + val httpAddress = getNodeManagerHttpAddress + val user = Utils.getCurrentUserName() + val httpScheme = getYarnHttpScheme(yarnConf) + + Some(Map( + "HTTP_SCHEME" -> httpScheme, + "NODE_HTTP_ADDRESS" -> httpAddress, + "CLUSTER_ID" -> clusterId.getOrElse(""), + "CONTAINER_ID" -> ConverterUtils.toString(containerId), + "USER" -> user, + "LOG_FILES" -> "stderr,stdout" + )) + } catch { + case e: Exception => + logInfo("Error while retrieving executor attributes - executor logs will not be replaced " + + "with custom log pattern", e) + None + } + } + + private def getContainerId(container: Option[Container]): ContainerId = container match { + case Some(c) => c.getId + case None => YarnSparkHadoopUtil.getContainerId + } + + private def getClusterId(yarnConf: YarnConfiguration): Option[String] = { + try { + Some(YarnConfiguration.getClusterId(yarnConf)) + } catch { + case _: HadoopIllegalArgumentException => None + } + } + + private def getNodeManagerHttpAddress: String = { + System.getenv(Environment.NM_HOST.name()) + ":" + + System.getenv(Environment.NM_HTTP_PORT.name()) + } + + private def getYarnHttpScheme(yarnConf: YarnConfiguration): String = { + // lookup appropriate http scheme for container log urls + val yarnHttpPolicy = yarnConf.get( + YarnConfiguration.YARN_HTTP_POLICY_KEY, + YarnConfiguration.YARN_HTTP_POLICY_DEFAULT + ) + if (yarnHttpPolicy == "HTTPS_ONLY") "https://" else "http://" + } +} From e4db8c502d9c1d628b2310ffbc78a630e74c5a4b Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Sat, 26 Jan 2019 10:42:03 +0900 Subject: [PATCH 23/28] Address review comments from vanzin --- ...pply_incomplete_app_false_expectation.json | 131 ------------------ ...apply_incomplete_app_true_expectation.json | 131 ------------------ ...pply_incomplete_app_false_expectation.json | 121 ---------------- ...apply_incomplete_app_true_expectation.json | 131 ------------------ .../history/FsHistoryProviderSuite.scala | 46 +++--- dev/.rat-excludes | 2 - docs/running-on-yarn.md | 18 ++- .../spark/deploy/yarn/ExecutorRunnable.scala | 6 +- .../cluster/YarnClusterSchedulerBackend.scala | 6 +- ...er.scala => YarnContainerInfoHelper.scala} | 32 +++-- .../spark/deploy/yarn/YarnClusterSuite.scala | 33 ++--- 11 files changed, 81 insertions(+), 576 deletions(-) delete mode 100644 core/src/test/resources/HistoryServerExpectations/completed_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_false_expectation.json delete mode 100644 core/src/test/resources/HistoryServerExpectations/completed_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_true_expectation.json delete mode 100644 core/src/test/resources/HistoryServerExpectations/incomplete_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_false_expectation.json delete mode 100644 core/src/test/resources/HistoryServerExpectations/incomplete_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_true_expectation.json rename resource-managers/yarn/src/main/scala/org/apache/spark/util/{YarnExecutorHelper.scala => YarnContainerInfoHelper.scala} (77%) diff --git a/core/src/test/resources/HistoryServerExpectations/completed_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_false_expectation.json b/core/src/test/resources/HistoryServerExpectations/completed_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_false_expectation.json deleted file mode 100644 index a7535680b4704..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/completed_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_false_expectation.json +++ /dev/null @@ -1,131 +0,0 @@ -[ - { - "id":"driver", - "hostPort":"node2:40675", - "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":428762726, - "addTime":"2019-01-18T05:14:05.481GMT", - "executorLogs":{ - "stderr" : "http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000001/stderr", - "stdout" : "http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000001/stdout" - }, - "memoryMetrics":{ - "usedOnHeapStorageMemory":0, - "usedOffHeapStorageMemory":0, - "totalOnHeapStorageMemory":428762726, - "totalOffHeapStorageMemory":0 - }, - "blacklistedInStages":[ - - ], - "attributes" : { - "NODE_HTTP_ADDRESS" : "node2:8042", - "USER" : "spark", - "LOG_FILES" : "stderr,stdout", - "CLUSTER_ID" : "yarn-cluster", - "HTTP_SCHEME" : "http://", - "CONTAINER_ID" : "container_e01_1547723113049_0005_01_000001" - } - }, - { - "id":"2", - "hostPort":"node4:44875", - "isActive":true, - "rddBlocks":0, - "memoryUsed":0, - "diskUsed":0, - "totalCores":1, - "maxTasks":1, - "activeTasks":0, - "failedTasks":0, - "completedTasks":0, - "totalTasks":0, - "totalDuration":0, - "totalGCTime":0, - "totalInputBytes":0, - "totalShuffleRead":0, - "totalShuffleWrite":0, - "isBlacklisted":false, - "maxMemory":384093388, - "addTime":"2019-01-18T05:14:15.343GMT", - "executorLogs":{ - "stderr":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000003/stderr", - "stdout":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000003/stdout" - }, - "memoryMetrics":{ - "usedOnHeapStorageMemory":0, - "usedOffHeapStorageMemory":0, - "totalOnHeapStorageMemory":384093388, - "totalOffHeapStorageMemory":0 - }, - "blacklistedInStages":[ - - ], - "attributes":{ - "NODE_HTTP_ADDRESS":"node4:8042", - "USER":"spark", - "LOG_FILES":"stderr,stdout", - "CLUSTER_ID":"yarn-cluster", - "HTTP_SCHEME":"http://", - "CONTAINER_ID":"container_e01_1547723113049_0005_01_000003" - } - }, - { - "id":"1", - "hostPort":"node3:35087", - "isActive":true, - "rddBlocks":0, - "memoryUsed":0, - "diskUsed":0, - "totalCores":1, - "maxTasks":1, - "activeTasks":0, - "failedTasks":0, - "completedTasks":1, - "totalTasks":1, - "totalDuration":4025, - "totalGCTime":159, - "totalInputBytes":73, - "totalShuffleRead":0, - "totalShuffleWrite":0, - "isBlacklisted":false, - "maxMemory":384093388, - "addTime":"2019-01-18T05:14:14.209GMT", - "executorLogs":{ - "stderr":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000002/stderr", - "stdout":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000002/stdout" - }, - "memoryMetrics":{ - "usedOnHeapStorageMemory":0, - "usedOffHeapStorageMemory":0, - "totalOnHeapStorageMemory":384093388, - "totalOffHeapStorageMemory":0 - }, - "blacklistedInStages":[ - - ], - "attributes":{ - "NODE_HTTP_ADDRESS":"node3:8042", - "USER":"spark", - "LOG_FILES":"stderr,stdout", - "CLUSTER_ID":"yarn-cluster", - "HTTP_SCHEME":"http://", - "CONTAINER_ID":"container_e01_1547723113049_0005_01_000002" - } - } -] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/completed_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_true_expectation.json b/core/src/test/resources/HistoryServerExpectations/completed_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_true_expectation.json deleted file mode 100644 index a7535680b4704..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/completed_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_true_expectation.json +++ /dev/null @@ -1,131 +0,0 @@ -[ - { - "id":"driver", - "hostPort":"node2:40675", - "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":428762726, - "addTime":"2019-01-18T05:14:05.481GMT", - "executorLogs":{ - "stderr" : "http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000001/stderr", - "stdout" : "http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000001/stdout" - }, - "memoryMetrics":{ - "usedOnHeapStorageMemory":0, - "usedOffHeapStorageMemory":0, - "totalOnHeapStorageMemory":428762726, - "totalOffHeapStorageMemory":0 - }, - "blacklistedInStages":[ - - ], - "attributes" : { - "NODE_HTTP_ADDRESS" : "node2:8042", - "USER" : "spark", - "LOG_FILES" : "stderr,stdout", - "CLUSTER_ID" : "yarn-cluster", - "HTTP_SCHEME" : "http://", - "CONTAINER_ID" : "container_e01_1547723113049_0005_01_000001" - } - }, - { - "id":"2", - "hostPort":"node4:44875", - "isActive":true, - "rddBlocks":0, - "memoryUsed":0, - "diskUsed":0, - "totalCores":1, - "maxTasks":1, - "activeTasks":0, - "failedTasks":0, - "completedTasks":0, - "totalTasks":0, - "totalDuration":0, - "totalGCTime":0, - "totalInputBytes":0, - "totalShuffleRead":0, - "totalShuffleWrite":0, - "isBlacklisted":false, - "maxMemory":384093388, - "addTime":"2019-01-18T05:14:15.343GMT", - "executorLogs":{ - "stderr":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000003/stderr", - "stdout":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000003/stdout" - }, - "memoryMetrics":{ - "usedOnHeapStorageMemory":0, - "usedOffHeapStorageMemory":0, - "totalOnHeapStorageMemory":384093388, - "totalOffHeapStorageMemory":0 - }, - "blacklistedInStages":[ - - ], - "attributes":{ - "NODE_HTTP_ADDRESS":"node4:8042", - "USER":"spark", - "LOG_FILES":"stderr,stdout", - "CLUSTER_ID":"yarn-cluster", - "HTTP_SCHEME":"http://", - "CONTAINER_ID":"container_e01_1547723113049_0005_01_000003" - } - }, - { - "id":"1", - "hostPort":"node3:35087", - "isActive":true, - "rddBlocks":0, - "memoryUsed":0, - "diskUsed":0, - "totalCores":1, - "maxTasks":1, - "activeTasks":0, - "failedTasks":0, - "completedTasks":1, - "totalTasks":1, - "totalDuration":4025, - "totalGCTime":159, - "totalInputBytes":73, - "totalShuffleRead":0, - "totalShuffleWrite":0, - "isBlacklisted":false, - "maxMemory":384093388, - "addTime":"2019-01-18T05:14:14.209GMT", - "executorLogs":{ - "stderr":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000002/stderr", - "stdout":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000002/stdout" - }, - "memoryMetrics":{ - "usedOnHeapStorageMemory":0, - "usedOffHeapStorageMemory":0, - "totalOnHeapStorageMemory":384093388, - "totalOffHeapStorageMemory":0 - }, - "blacklistedInStages":[ - - ], - "attributes":{ - "NODE_HTTP_ADDRESS":"node3:8042", - "USER":"spark", - "LOG_FILES":"stderr,stdout", - "CLUSTER_ID":"yarn-cluster", - "HTTP_SCHEME":"http://", - "CONTAINER_ID":"container_e01_1547723113049_0005_01_000002" - } - } -] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/incomplete_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_false_expectation.json b/core/src/test/resources/HistoryServerExpectations/incomplete_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_false_expectation.json deleted file mode 100644 index d4bd46c36bc4f..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/incomplete_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_false_expectation.json +++ /dev/null @@ -1,121 +0,0 @@ -[ { - "id" : "driver", - "hostPort" : "node2:40675", - "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" : 428762726, - "addTime" : "2019-01-18T05:14:05.481GMT", - "executorLogs" : { - "stdout" : "http://node2:8042/node/containerlogs/container_e01_1547723113049_0005_01_000001/spark/stdout?start=-4096", - "stderr" : "http://node2:8042/node/containerlogs/container_e01_1547723113049_0005_01_000001/spark/stderr?start=-4096" - }, - "memoryMetrics" : { - "usedOnHeapStorageMemory" : 0, - "usedOffHeapStorageMemory" : 0, - "totalOnHeapStorageMemory" : 428762726, - "totalOffHeapStorageMemory" : 0 - }, - "blacklistedInStages" : [ ], - "attributes" : { - "NODE_HTTP_ADDRESS" : "node2:8042", - "USER" : "spark", - "LOG_FILES" : "stderr,stdout", - "CLUSTER_ID" : "yarn-cluster", - "HTTP_SCHEME" : "http://", - "CONTAINER_ID" : "container_e01_1547723113049_0005_01_000001" - } -}, { - "id" : "2", - "hostPort" : "node4:44875", - "isActive" : true, - "rddBlocks" : 0, - "memoryUsed" : 0, - "diskUsed" : 0, - "totalCores" : 1, - "maxTasks" : 1, - "activeTasks" : 0, - "failedTasks" : 0, - "completedTasks" : 0, - "totalTasks" : 0, - "totalDuration" : 0, - "totalGCTime" : 0, - "totalInputBytes" : 0, - "totalShuffleRead" : 0, - "totalShuffleWrite" : 0, - "isBlacklisted" : false, - "maxMemory" : 384093388, - "addTime" : "2019-01-18T05:14:15.343GMT", - "executorLogs" : { - "stdout" : "http://node4:8042/node/containerlogs/container_e01_1547723113049_0005_01_000003/spark/stdout?start=-4096", - "stderr" : "http://node4:8042/node/containerlogs/container_e01_1547723113049_0005_01_000003/spark/stderr?start=-4096" - }, - "memoryMetrics" : { - "usedOnHeapStorageMemory" : 0, - "usedOffHeapStorageMemory" : 0, - "totalOnHeapStorageMemory" : 384093388, - "totalOffHeapStorageMemory" : 0 - }, - "blacklistedInStages" : [ ], - "attributes" : { - "NODE_HTTP_ADDRESS" : "node4:8042", - "USER" : "spark", - "LOG_FILES" : "stderr,stdout", - "CLUSTER_ID" : "yarn-cluster", - "HTTP_SCHEME" : "http://", - "CONTAINER_ID" : "container_e01_1547723113049_0005_01_000003" - } -}, { - "id" : "1", - "hostPort" : "node3:35087", - "isActive" : true, - "rddBlocks" : 0, - "memoryUsed" : 0, - "diskUsed" : 0, - "totalCores" : 1, - "maxTasks" : 1, - "activeTasks" : 0, - "failedTasks" : 0, - "completedTasks" : 1, - "totalTasks" : 1, - "totalDuration" : 4025, - "totalGCTime" : 159, - "totalInputBytes" : 73, - "totalShuffleRead" : 0, - "totalShuffleWrite" : 0, - "isBlacklisted" : false, - "maxMemory" : 384093388, - "addTime" : "2019-01-18T05:14:14.209GMT", - "executorLogs" : { - "stdout" : "http://node3:8042/node/containerlogs/container_e01_1547723113049_0005_01_000002/spark/stdout?start=-4096", - "stderr" : "http://node3:8042/node/containerlogs/container_e01_1547723113049_0005_01_000002/spark/stderr?start=-4096" - }, - "memoryMetrics" : { - "usedOnHeapStorageMemory" : 0, - "usedOffHeapStorageMemory" : 0, - "totalOnHeapStorageMemory" : 384093388, - "totalOffHeapStorageMemory" : 0 - }, - "blacklistedInStages" : [ ], - "attributes" : { - "NODE_HTTP_ADDRESS" : "node3:8042", - "USER" : "spark", - "LOG_FILES" : "stderr,stdout", - "CLUSTER_ID" : "yarn-cluster", - "HTTP_SCHEME" : "http://", - "CONTAINER_ID" : "container_e01_1547723113049_0005_01_000002" - } -} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/incomplete_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_true_expectation.json b/core/src/test/resources/HistoryServerExpectations/incomplete_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_true_expectation.json deleted file mode 100644 index a7535680b4704..0000000000000 --- a/core/src/test/resources/HistoryServerExpectations/incomplete_app_executor_list_json_apply_custom_log_urls_apply_incomplete_app_true_expectation.json +++ /dev/null @@ -1,131 +0,0 @@ -[ - { - "id":"driver", - "hostPort":"node2:40675", - "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":428762726, - "addTime":"2019-01-18T05:14:05.481GMT", - "executorLogs":{ - "stderr" : "http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000001/stderr", - "stdout" : "http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000001/stdout" - }, - "memoryMetrics":{ - "usedOnHeapStorageMemory":0, - "usedOffHeapStorageMemory":0, - "totalOnHeapStorageMemory":428762726, - "totalOffHeapStorageMemory":0 - }, - "blacklistedInStages":[ - - ], - "attributes" : { - "NODE_HTTP_ADDRESS" : "node2:8042", - "USER" : "spark", - "LOG_FILES" : "stderr,stdout", - "CLUSTER_ID" : "yarn-cluster", - "HTTP_SCHEME" : "http://", - "CONTAINER_ID" : "container_e01_1547723113049_0005_01_000001" - } - }, - { - "id":"2", - "hostPort":"node4:44875", - "isActive":true, - "rddBlocks":0, - "memoryUsed":0, - "diskUsed":0, - "totalCores":1, - "maxTasks":1, - "activeTasks":0, - "failedTasks":0, - "completedTasks":0, - "totalTasks":0, - "totalDuration":0, - "totalGCTime":0, - "totalInputBytes":0, - "totalShuffleRead":0, - "totalShuffleWrite":0, - "isBlacklisted":false, - "maxMemory":384093388, - "addTime":"2019-01-18T05:14:15.343GMT", - "executorLogs":{ - "stderr":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000003/stderr", - "stdout":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000003/stdout" - }, - "memoryMetrics":{ - "usedOnHeapStorageMemory":0, - "usedOffHeapStorageMemory":0, - "totalOnHeapStorageMemory":384093388, - "totalOffHeapStorageMemory":0 - }, - "blacklistedInStages":[ - - ], - "attributes":{ - "NODE_HTTP_ADDRESS":"node4:8042", - "USER":"spark", - "LOG_FILES":"stderr,stdout", - "CLUSTER_ID":"yarn-cluster", - "HTTP_SCHEME":"http://", - "CONTAINER_ID":"container_e01_1547723113049_0005_01_000003" - } - }, - { - "id":"1", - "hostPort":"node3:35087", - "isActive":true, - "rddBlocks":0, - "memoryUsed":0, - "diskUsed":0, - "totalCores":1, - "maxTasks":1, - "activeTasks":0, - "failedTasks":0, - "completedTasks":1, - "totalTasks":1, - "totalDuration":4025, - "totalGCTime":159, - "totalInputBytes":73, - "totalShuffleRead":0, - "totalShuffleWrite":0, - "isBlacklisted":false, - "maxMemory":384093388, - "addTime":"2019-01-18T05:14:14.209GMT", - "executorLogs":{ - "stderr":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000002/stderr", - "stdout":"http://newhost:9999/logs/clusters/yarn-cluster/users/spark/containers/container_e01_1547723113049_0005_01_000002/stdout" - }, - "memoryMetrics":{ - "usedOnHeapStorageMemory":0, - "usedOffHeapStorageMemory":0, - "totalOnHeapStorageMemory":384093388, - "totalOffHeapStorageMemory":0 - }, - "blacklistedInStages":[ - - ], - "attributes":{ - "NODE_HTTP_ADDRESS":"node3:8042", - "USER":"spark", - "LOG_FILES":"stderr,stdout", - "CLUSTER_ID":"yarn-cluster", - "HTTP_SCHEME":"http://", - "CONTAINER_ID":"container_e01_1547723113049_0005_01_000002" - } - } -] \ No newline at end of file diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 7cf0703868893..bce17334a47bf 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -305,8 +305,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc test("custom log urls, including FILE_NAME") { val conf = createTestConf() - .set(CUSTOM_EXECUTOR_LOG_URL, "http://newhost:9999/logs/clusters/{{CLUSTER_ID}}" + - "/users/{{USER}}/containers/{{CONTAINER_ID}}/{{FILE_NAME}}") + .set(CUSTOM_EXECUTOR_LOG_URL, getCustomExecutorLogUrl(includeFileName = true)) // some of available attributes are not used in pattern which should be OK @@ -315,8 +314,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc val expected: Map[ExecutorInfo, Map[String, String]] = executorInfos.map { execInfo => val attr = execInfo.attributes val newLogUrlMap = attr("LOG_FILES").split(",").map { file => - val newLogUrl = s"http://newhost:9999/logs/clusters/${attr("CLUSTER_ID")}" + - s"/users/${attr("USER")}/containers/${attr("CONTAINER_ID")}/$file" + val newLogUrl = getExpectedExecutorLogUrl(attr, Some(file)) file -> newLogUrl }.toMap @@ -328,8 +326,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc test("custom log urls, excluding FILE_NAME") { val conf = createTestConf() - .set(CUSTOM_EXECUTOR_LOG_URL, "http://newhost:9999/logs/clusters/{{CLUSTER_ID}}" + - "/users/{{USER}}/containers/{{CONTAINER_ID}}") + .set(CUSTOM_EXECUTOR_LOG_URL, getCustomExecutorLogUrl(includeFileName = false)) // some of available attributes are not used in pattern which should be OK @@ -337,8 +334,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc val expected: Map[ExecutorInfo, Map[String, String]] = executorInfos.map { execInfo => val attr = execInfo.attributes - val newLogUrl = s"http://newhost:9999/logs/clusters/${attr("CLUSTER_ID")}" + - s"/users/${attr("USER")}/containers/${attr("CONTAINER_ID")}" + val newLogUrl = getExpectedExecutorLogUrl(attr, None) execInfo -> Map("log" -> newLogUrl) }.toMap @@ -351,8 +347,8 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc // which Spark will fail back to provide origin log url with warning log. val conf = createTestConf() - .set(CUSTOM_EXECUTOR_LOG_URL, "http://newhost:9999/logs/clusters/{{CLUSTER_ID}}" + - "/users/{{USER}}/containers/{{CONTAINER_ID}}/{{NON_EXISTING}}/{{FILE_NAME}}") + .set(CUSTOM_EXECUTOR_LOG_URL, getCustomExecutorLogUrl(includeFileName = true) + + "/{{NON_EXISTING}}") val executorInfos = (1 to 5).map(createTestExecutorInfo("app1", "user1", _)) @@ -367,8 +363,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc // For this case Spark will fail back to provide origin log url with warning log. val conf = createTestConf() - .set(CUSTOM_EXECUTOR_LOG_URL, "http://newhost:9999/logs/clusters/{{CLUSTER_ID}}" + - "/users/{{USER}}/containers/{{CONTAINER_ID}}/{{NON_EXISTING}}/{{FILE_NAME}}") + .set(CUSTOM_EXECUTOR_LOG_URL, getCustomExecutorLogUrl(includeFileName = true)) val executorInfos = (1 to 5).map( createTestExecutorInfo("app1", "user1", _, includingLogFiles = false)) @@ -382,8 +377,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc test("custom log urls, app not finished, applyIncompleteApplication: true") { val conf = createTestConf() - .set(CUSTOM_EXECUTOR_LOG_URL, "http://newhost:9999/logs/clusters/{{CLUSTER_ID}}" + - "/users/{{USER}}/containers/{{CONTAINER_ID}}/{{FILE_NAME}}") + .set(CUSTOM_EXECUTOR_LOG_URL, getCustomExecutorLogUrl(includeFileName = true)) .set(APPLY_CUSTOM_EXECUTOR_LOG_URL_TO_INCOMPLETE_APP, true) // ensure custom log urls are applied to incomplete application @@ -393,8 +387,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc val expected: Map[ExecutorInfo, Map[String, String]] = executorInfos.map { execInfo => val attr = execInfo.attributes val newLogUrlMap = attr("LOG_FILES").split(",").map { file => - val newLogUrl = s"http://newhost:9999/logs/clusters/${attr("CLUSTER_ID")}" + - s"/users/${attr("USER")}/containers/${attr("CONTAINER_ID")}/$file" + val newLogUrl = getExpectedExecutorLogUrl(attr, Some(file)) file -> newLogUrl }.toMap @@ -406,8 +399,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc test("custom log urls, app not finished, applyIncompleteApplication: false") { val conf = createTestConf() - .set(CUSTOM_EXECUTOR_LOG_URL, "http://newhost:9999/logs/clusters/{{CLUSTER_ID}}" + - "/users/{{USER}}/containers/{{CONTAINER_ID}}/{{FILE_NAME}}") + .set(CUSTOM_EXECUTOR_LOG_URL, getCustomExecutorLogUrl(includeFileName = true)) .set(APPLY_CUSTOM_EXECUTOR_LOG_URL_TO_INCOMPLETE_APP, false) // ensure custom log urls are NOT applied to incomplete application @@ -421,6 +413,24 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc testHandlingExecutorLogUrl(conf, expected, isCompletedApp = false) } + private def getCustomExecutorLogUrl(includeFileName: Boolean): String = { + val baseUrl = "http://newhost:9999/logs/clusters/{{CLUSTER_ID}}/users/{{USER}}/containers/" + + "{{CONTAINER_ID}}" + if (includeFileName) baseUrl + "/{{FILE_NAME}}" else baseUrl + } + + private def getExpectedExecutorLogUrl( + attributes: Map[String, String], + fileName: Option[String]): String = { + val baseUrl = s"http://newhost:9999/logs/clusters/${attributes("CLUSTER_ID")}" + + s"/users/${attributes("USER")}/containers/${attributes("CONTAINER_ID")}" + + fileName match { + case Some(file) => baseUrl + s"/$file" + case None => baseUrl + } + } + private def testHandlingExecutorLogUrl( conf: SparkConf, expectedLogUrlMap: Map[ExecutorInfo, Map[String, String]], diff --git a/dev/.rat-excludes b/dev/.rat-excludes index 9da5d1a1c8923..8239cbc3a381c 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -83,8 +83,6 @@ app-20161116163331-0000 application_1516285256255_0012 application_1506645932520_24630151 application_1538416563558_0014 -application_1547723113049_0005 -application_1547723113049_0006 stat local-1422981759269 local-1422981780767 diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 036cc8fd1599b..3707b48cdf2cf 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -481,8 +481,16 @@ To use a custom metrics.properties for the application master and executors, upd `http://`/`https://` according to YARN HTTP policy. (Configured via `yarn.http.policy`) - {{NODE_HTTP_ADDRESS}} - The "host:port" of node where container was run. + {{NODE_HOST}} + The "host" of node where container was run. + + + {{NODE_PORT}} + The "port" of node manager where container was run. + + + {{NODE_HTTP_PORT}} + The "port" of node manager's http server where container was run. {{CLUSTER_ID}} @@ -502,6 +510,12 @@ To use a custom metrics.properties for the application master and executors, upd +For example, suppose you would like to point log url link to Job History Server directly instead of let NodeManager http server redirects it, you can configure `spark.history.custom.executor.log.url` as below: + +`{{HTTP_SCHEME}}:/jobhistory/logs/{{NODE_HOST}}:{{NODE_PORT}}/{{CONTAINER_ID}}/{{CONTAINER_ID}}/{{USER}}/{{FILE_NAME}}?start=-4096` + +NOTE: you need to replace `` and `` with actual value. + # Important notes - Whether core requests are honored in scheduling decisions depends on which scheduler is in use and how it is configured. diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index ce5c097c334f1..a0e1bfd6e2b72 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -40,7 +40,7 @@ import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.network.util.JavaUtils -import org.apache.spark.util.{Utils, YarnExecutorHelper} +import org.apache.spark.util.{Utils, YarnContainerInfoHelper} private[yarn] class ExecutorRunnable( container: Option[Container], @@ -243,13 +243,13 @@ private[yarn] class ExecutorRunnable( // Add log urls, as well as executor attributes container.foreach { c => - YarnExecutorHelper.getLogUrls(conf, Some(c)).foreach { m => + YarnContainerInfoHelper.getLogUrls(conf, Some(c)).foreach { m => m.foreach { case (fileName, url) => env("SPARK_LOG_URL_" + fileName.toUpperCase(Locale.ROOT)) = url } } - YarnExecutorHelper.getAttributes(conf, Some(c)).foreach { m => + YarnContainerInfoHelper.getAttributes(conf, Some(c)).foreach { m => m.foreach { case (attr, value) => env("SPARK_EXECUTOR_ATTRIBUTE_" + attr.toUpperCase(Locale.ROOT)) = value } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala index 58acdeb0ed64e..b5575a10a05a0 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala @@ -20,7 +20,7 @@ package org.apache.spark.scheduler.cluster import org.apache.spark.SparkContext import org.apache.spark.deploy.yarn.ApplicationMaster import org.apache.spark.scheduler.TaskSchedulerImpl -import org.apache.spark.util.YarnExecutorHelper +import org.apache.spark.util.YarnContainerInfoHelper private[spark] class YarnClusterSchedulerBackend( scheduler: TaskSchedulerImpl, @@ -36,10 +36,10 @@ private[spark] class YarnClusterSchedulerBackend( } override def getDriverLogUrls: Option[Map[String, String]] = { - YarnExecutorHelper.getLogUrls(sc.hadoopConfiguration, container = None) + YarnContainerInfoHelper.getLogUrls(sc.hadoopConfiguration, container = None) } override def getDriverAttributes: Option[Map[String, String]] = { - YarnExecutorHelper.getAttributes(sc.hadoopConfiguration, container = None) + YarnContainerInfoHelper.getAttributes(sc.hadoopConfiguration, container = None) } } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/util/YarnExecutorHelper.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/util/YarnContainerInfoHelper.scala similarity index 77% rename from resource-managers/yarn/src/main/scala/org/apache/spark/util/YarnExecutorHelper.scala rename to resource-managers/yarn/src/main/scala/org/apache/spark/util/YarnContainerInfoHelper.scala index 992b4102e98b9..5be062e212151 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/util/YarnExecutorHelper.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/util/YarnContainerInfoHelper.scala @@ -27,7 +27,7 @@ import org.apache.hadoop.yarn.util.ConverterUtils import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil import org.apache.spark.internal.Logging -private[spark] object YarnExecutorHelper extends Logging { +private[spark] object YarnContainerInfoHelper extends Logging { def getLogUrls( conf: Configuration, container: Option[Container]): Option[Map[String, String]] = { @@ -35,11 +35,12 @@ private[spark] object YarnExecutorHelper extends Logging { val yarnConf = new YarnConfiguration(conf) val containerId = getContainerId(container) - val httpAddress = getNodeManagerHttpAddress val user = Utils.getCurrentUserName() val httpScheme = getYarnHttpScheme(yarnConf) + val host = getNodeManagerHost + val httpPort = getNodeManagerHttpPort - val baseUrl = s"$httpScheme$httpAddress/node/containerlogs/$containerId/$user" + val baseUrl = s"$httpScheme$host:$httpPort/node/containerlogs/$containerId/$user" logDebug(s"Base URL for logs: $baseUrl") Some(Map( @@ -60,13 +61,17 @@ private[spark] object YarnExecutorHelper extends Logging { val clusterId = getClusterId(yarnConf) val containerId = getContainerId(container) - val httpAddress = getNodeManagerHttpAddress + val host = getNodeManagerHost + val port = getNodeManagerPort + val httpPort = getNodeManagerHttpPort val user = Utils.getCurrentUserName() val httpScheme = getYarnHttpScheme(yarnConf) Some(Map( "HTTP_SCHEME" -> httpScheme, - "NODE_HTTP_ADDRESS" -> httpAddress, + "NODE_HOST" -> host, + "NODE_PORT" -> port.toString, + "NODE_HTTP_PORT" -> httpPort.toString, "CLUSTER_ID" -> clusterId.getOrElse(""), "CONTAINER_ID" -> ConverterUtils.toString(containerId), "USER" -> user, @@ -80,12 +85,12 @@ private[spark] object YarnExecutorHelper extends Logging { } } - private def getContainerId(container: Option[Container]): ContainerId = container match { + def getContainerId(container: Option[Container]): ContainerId = container match { case Some(c) => c.getId case None => YarnSparkHadoopUtil.getContainerId } - private def getClusterId(yarnConf: YarnConfiguration): Option[String] = { + def getClusterId(yarnConf: YarnConfiguration): Option[String] = { try { Some(YarnConfiguration.getClusterId(yarnConf)) } catch { @@ -93,12 +98,7 @@ private[spark] object YarnExecutorHelper extends Logging { } } - private def getNodeManagerHttpAddress: String = { - System.getenv(Environment.NM_HOST.name()) + ":" + - System.getenv(Environment.NM_HTTP_PORT.name()) - } - - private def getYarnHttpScheme(yarnConf: YarnConfiguration): String = { + def getYarnHttpScheme(yarnConf: YarnConfiguration): String = { // lookup appropriate http scheme for container log urls val yarnHttpPolicy = yarnConf.get( YarnConfiguration.YARN_HTTP_POLICY_KEY, @@ -106,4 +106,10 @@ private[spark] object YarnExecutorHelper extends Logging { ) if (yarnHttpPolicy == "HTTPS_ONLY") "https://" else "http://" } + + def getNodeManagerHost: String = System.getenv(Environment.NM_HOST.name()) + + def getNodeManagerPort: Int = System.getenv(Environment.NM_PORT.name()).toInt + + def getNodeManagerHttpPort: Int = System.getenv(Environment.NM_HTTP_PORT.name()).toInt } diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index 0a6753715a859..c15195278eca5 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -42,11 +42,10 @@ import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.internal.config.UI._ import org.apache.spark.launcher._ -import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationStart, - SparkListenerExecutorAdded} +import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationStart, SparkListenerExecutorAdded} import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.tags.ExtendedYarnTest -import org.apache.spark.util.Utils +import org.apache.spark.util.{Utils, YarnContainerInfoHelper} /** * Integration tests for YARN; these tests use a mini Yarn cluster to run Spark-on-YARN @@ -469,31 +468,23 @@ private object YarnClusterDriver extends Logging with Matchers { } val yarnConf = new YarnConfiguration(sc.hadoopConfiguration) - val httpAddress = System.getenv(ApplicationConstants.Environment.NM_HOST.name()) + - ":" + System.getenv(ApplicationConstants.Environment.NM_HTTP_PORT.name()) - - // lookup appropriate http scheme for container log urls - val yarnHttpPolicy = yarnConf.get( - YarnConfiguration.YARN_HTTP_POLICY_KEY, - YarnConfiguration.YARN_HTTP_POLICY_DEFAULT - ) - val httpScheme = if (yarnHttpPolicy == "HTTPS_ONLY") "https://" else "http://" - - val containerId = YarnSparkHadoopUtil.getContainerId + val host = YarnContainerInfoHelper.getNodeManagerHost + val port = YarnContainerInfoHelper.getNodeManagerPort + val httpPort = YarnContainerInfoHelper.getNodeManagerHttpPort + val httpScheme = YarnContainerInfoHelper.getYarnHttpScheme(yarnConf) + val containerId = YarnContainerInfoHelper.getContainerId(container = None) val user = Utils.getCurrentUserName() + val clusterId: Option[String] = YarnContainerInfoHelper.getClusterId(yarnConf) + assert(urlStr.endsWith(s"/node/containerlogs/$containerId/$user/stderr?start=-4096")) assert(listener.driverAttributes.nonEmpty) val driverAttributes = listener.driverAttributes.get - val clusterId: Option[String] = try { - Some(YarnConfiguration.getClusterId(yarnConf)) - } catch { - case _: HadoopIllegalArgumentException => None - } - val expectationAttributes = Map( "HTTP_SCHEME" -> httpScheme, - "NODE_HTTP_ADDRESS" -> httpAddress, + "NODE_HOST" -> host, + "NODE_PORT" -> port, + "NODE_HTTP_PORT" -> httpPort, "CLUSTER_ID" -> clusterId.getOrElse(""), "CONTAINER_ID" -> ConverterUtils.toString(containerId), "USER" -> user, From 0acfb20eeb6b72a0c3f2ad0f88916fc10579a437 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Sun, 27 Jan 2019 06:41:51 +0900 Subject: [PATCH 24/28] Fix a silly bug --- .../scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index c15195278eca5..a3c077bac8719 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -483,8 +483,8 @@ private object YarnClusterDriver extends Logging with Matchers { val expectationAttributes = Map( "HTTP_SCHEME" -> httpScheme, "NODE_HOST" -> host, - "NODE_PORT" -> port, - "NODE_HTTP_PORT" -> httpPort, + "NODE_PORT" -> port.toString, + "NODE_HTTP_PORT" -> httpPort.toString, "CLUSTER_ID" -> clusterId.getOrElse(""), "CONTAINER_ID" -> ConverterUtils.toString(containerId), "USER" -> user, From 954b17a78b7b548c067fa7b6b0723eec154e89e0 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Tue, 29 Jan 2019 05:21:46 +0900 Subject: [PATCH 25/28] Address review comments --- .../history/HistoryAppStatusStore.scala | 2 +- docs/monitoring.md | 8 +++---- docs/running-on-yarn.md | 8 +++---- .../spark/util/YarnContainerInfoHelper.scala | 23 ++++++------------- .../spark/deploy/yarn/YarnClusterSuite.scala | 6 ++--- 5 files changed, 19 insertions(+), 28 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala index 6e9c2ed8a74b5..751382c9a2e56 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala @@ -110,7 +110,7 @@ private[spark] class HistoryAppStatusStore( allAttributes: Set[String]): Unit = { if (informedForMissingAttributes.compareAndSet(false, true)) { logInfo(s"Fail to renew executor log urls: $reason. Required: $allPatterns / " + - s"available: $allAttributes. Failing back to show app's original log urls.") + s"available: $allAttributes. Falling back to show app's original log urls.") } } diff --git a/docs/monitoring.md b/docs/monitoring.md index 63b71fcc6f269..1e292d2111f3b 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -252,8 +252,8 @@ Security options for the Spark History Server are covered more detail in the spark.history.custom.executor.log.url (none) - Specifies custom spark executor log url for supporting external log service instead of using cluster - managers' application log urls in the history server. Spark will support some path variables via patterns + Specifies custom spark executor log URL for supporting external log service instead of using cluster + managers' application log URLs in the history server. Spark will support some path variables via patterns which can vary on cluster manager. Please check the documentation for your cluster manager to see which patterns are supported, if any. This configuration has no effect on a live application, it only affects the history server. @@ -265,8 +265,8 @@ Security options for the Spark History Server are covered more detail in the spark.history.custom.executor.log.url.applyIncompleteApplication false - Specifies whether to apply custom spark executor log url to incomplete applications as well. - If executor logs for running applications should be provided as origin log urls, set this to `false`. + Specifies whether to apply custom spark executor log URL to incomplete applications as well. + If executor logs for running applications should be provided as origin log URLs, set this to `false`. Please note that incomplete applications may include applications which didn't shutdown gracefully. Even this is set to `true`, this configuration has no effect on a live application, it only affects the history server. diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 3707b48cdf2cf..471acc2145514 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -481,15 +481,15 @@ To use a custom metrics.properties for the application master and executors, upd `http://`/`https://` according to YARN HTTP policy. (Configured via `yarn.http.policy`) - {{NODE_HOST}} + {{NM_HOST}} The "host" of node where container was run. - {{NODE_PORT}} + {{NM_PORT}} The "port" of node manager where container was run. - {{NODE_HTTP_PORT}} + {{NM_HTTP_PORT}} The "port" of node manager's http server where container was run. @@ -512,7 +512,7 @@ To use a custom metrics.properties for the application master and executors, upd For example, suppose you would like to point log url link to Job History Server directly instead of let NodeManager http server redirects it, you can configure `spark.history.custom.executor.log.url` as below: -`{{HTTP_SCHEME}}:/jobhistory/logs/{{NODE_HOST}}:{{NODE_PORT}}/{{CONTAINER_ID}}/{{CONTAINER_ID}}/{{USER}}/{{FILE_NAME}}?start=-4096` +`{{HTTP_SCHEME}}:/jobhistory/logs/{{NM_HOST}}:{{NM_PORT}}/{{CONTAINER_ID}}/{{CONTAINER_ID}}/{{USER}}/{{FILE_NAME}}?start=-4096` NOTE: you need to replace `` and `` with actual value. diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/util/YarnContainerInfoHelper.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/util/YarnContainerInfoHelper.scala index 5be062e212151..89db8e0528300 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/util/YarnContainerInfoHelper.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/util/YarnContainerInfoHelper.scala @@ -58,23 +58,14 @@ private[spark] object YarnContainerInfoHelper extends Logging { container: Option[Container]): Option[Map[String, String]] = { try { val yarnConf = new YarnConfiguration(conf) - - val clusterId = getClusterId(yarnConf) - val containerId = getContainerId(container) - val host = getNodeManagerHost - val port = getNodeManagerPort - val httpPort = getNodeManagerHttpPort - val user = Utils.getCurrentUserName() - val httpScheme = getYarnHttpScheme(yarnConf) - Some(Map( - "HTTP_SCHEME" -> httpScheme, - "NODE_HOST" -> host, - "NODE_PORT" -> port.toString, - "NODE_HTTP_PORT" -> httpPort.toString, - "CLUSTER_ID" -> clusterId.getOrElse(""), - "CONTAINER_ID" -> ConverterUtils.toString(containerId), - "USER" -> user, + "HTTP_SCHEME" -> getYarnHttpScheme(yarnConf), + "NM_HOST" -> getNodeManagerHost, + "NM_PORT" -> getNodeManagerPort.toString, + "NM_HTTP_PORT" -> getNodeManagerHttpPort.toString, + "CLUSTER_ID" -> getClusterId(yarnConf).getOrElse(""), + "CONTAINER_ID" -> ConverterUtils.toString(getContainerId(container)), + "USER" -> Utils.getCurrentUserName(), "LOG_FILES" -> "stderr,stdout" )) } catch { diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index a3c077bac8719..2810b7ffe3ab6 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -482,9 +482,9 @@ private object YarnClusterDriver extends Logging with Matchers { val driverAttributes = listener.driverAttributes.get val expectationAttributes = Map( "HTTP_SCHEME" -> httpScheme, - "NODE_HOST" -> host, - "NODE_PORT" -> port.toString, - "NODE_HTTP_PORT" -> httpPort.toString, + "NM_HOST" -> host, + "NM_PORT" -> port.toString, + "NM_HTTP_PORT" -> httpPort.toString, "CLUSTER_ID" -> clusterId.getOrElse(""), "CONTAINER_ID" -> ConverterUtils.toString(containerId), "USER" -> user, From 0df31ac6c32420e2ee0754f0cf6e996f9a9cc692 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Tue, 29 Jan 2019 05:42:45 +0900 Subject: [PATCH 26/28] More inlines... --- .../spark/deploy/yarn/YarnClusterSuite.scala | 15 +++++---------- 1 file changed, 5 insertions(+), 10 deletions(-) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index 2810b7ffe3ab6..d41b5e2278928 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -468,24 +468,19 @@ private object YarnClusterDriver extends Logging with Matchers { } val yarnConf = new YarnConfiguration(sc.hadoopConfiguration) - val host = YarnContainerInfoHelper.getNodeManagerHost - val port = YarnContainerInfoHelper.getNodeManagerPort - val httpPort = YarnContainerInfoHelper.getNodeManagerHttpPort - val httpScheme = YarnContainerInfoHelper.getYarnHttpScheme(yarnConf) val containerId = YarnContainerInfoHelper.getContainerId(container = None) val user = Utils.getCurrentUserName() - val clusterId: Option[String] = YarnContainerInfoHelper.getClusterId(yarnConf) assert(urlStr.endsWith(s"/node/containerlogs/$containerId/$user/stderr?start=-4096")) assert(listener.driverAttributes.nonEmpty) val driverAttributes = listener.driverAttributes.get val expectationAttributes = Map( - "HTTP_SCHEME" -> httpScheme, - "NM_HOST" -> host, - "NM_PORT" -> port.toString, - "NM_HTTP_PORT" -> httpPort.toString, - "CLUSTER_ID" -> clusterId.getOrElse(""), + "HTTP_SCHEME" -> YarnContainerInfoHelper.getYarnHttpScheme(yarnConf), + "NM_HOST" -> YarnContainerInfoHelper.getNodeManagerHost, + "NM_PORT" -> YarnContainerInfoHelper.getNodeManagerPort.toString, + "NM_HTTP_PORT" -> YarnContainerInfoHelper.getNodeManagerHttpPort.toString, + "CLUSTER_ID" -> YarnContainerInfoHelper.getClusterId(yarnConf).getOrElse(""), "CONTAINER_ID" -> ConverterUtils.toString(containerId), "USER" -> user, "LOG_FILES" -> "stderr,stdout") From 98b7b1653c6919750f2b47acfa324d617f87d5e3 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Tue, 29 Jan 2019 10:12:18 +0900 Subject: [PATCH 27/28] Remove unnecessary lines --- .../org/apache/spark/deploy/yarn/ExecutorRunnable.scala | 6 ------ 1 file changed, 6 deletions(-) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index a0e1bfd6e2b72..0b909d15c2faa 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -58,12 +58,6 @@ private[yarn] class ExecutorRunnable( var rpc: YarnRPC = YarnRPC.create(conf) var nmClient: NMClient = _ - val clusterId: Option[String] = try { - Some(YarnConfiguration.getClusterId(conf)) - } catch { - case _: HadoopIllegalArgumentException => None - } - def run(): Unit = { logDebug("Starting Executor Container") nmClient = NMClient.createNMClient() From 2d0480239a4eea655b95a521ac71ed44482acde7 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Tue, 29 Jan 2019 14:08:34 +0900 Subject: [PATCH 28/28] Fix a bug: we can't support NM_HOST / NM_PORT / NM_HTTP_PORT consistently * they're only available in container's env - cannot retrieve them outside of container process --- docs/running-on-yarn.md | 20 +++---------------- .../spark/util/YarnContainerInfoHelper.scala | 19 ++++++++---------- .../spark/deploy/yarn/YarnClusterSuite.scala | 4 +--- 3 files changed, 12 insertions(+), 31 deletions(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 471acc2145514..c3ebadb5c9cb3 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -478,19 +478,11 @@ To use a custom metrics.properties for the application master and executors, upd PatternMeaning {{HTTP_SCHEME}} - `http://`/`https://` according to YARN HTTP policy. (Configured via `yarn.http.policy`) + `http://` or `https://` according to YARN HTTP policy. (Configured via `yarn.http.policy`) - {{NM_HOST}} - The "host" of node where container was run. - - - {{NM_PORT}} - The "port" of node manager where container was run. - - - {{NM_HTTP_PORT}} - The "port" of node manager's http server where container was run. + {{NM_HTTP_ADDRESS}} + Http URI of the node on which the container is allocated. {{CLUSTER_ID}} @@ -510,12 +502,6 @@ To use a custom metrics.properties for the application master and executors, upd -For example, suppose you would like to point log url link to Job History Server directly instead of let NodeManager http server redirects it, you can configure `spark.history.custom.executor.log.url` as below: - -`{{HTTP_SCHEME}}:/jobhistory/logs/{{NM_HOST}}:{{NM_PORT}}/{{CONTAINER_ID}}/{{CONTAINER_ID}}/{{USER}}/{{FILE_NAME}}?start=-4096` - -NOTE: you need to replace `` and `` with actual value. - # Important notes - Whether core requests are honored in scheduling decisions depends on which scheduler is in use and how it is configured. diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/util/YarnContainerInfoHelper.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/util/YarnContainerInfoHelper.scala index 89db8e0528300..96350cdece556 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/util/YarnContainerInfoHelper.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/util/YarnContainerInfoHelper.scala @@ -37,10 +37,9 @@ private[spark] object YarnContainerInfoHelper extends Logging { val containerId = getContainerId(container) val user = Utils.getCurrentUserName() val httpScheme = getYarnHttpScheme(yarnConf) - val host = getNodeManagerHost - val httpPort = getNodeManagerHttpPort + val httpAddress = getNodeManagerHttpAddress(container) - val baseUrl = s"$httpScheme$host:$httpPort/node/containerlogs/$containerId/$user" + val baseUrl = s"$httpScheme$httpAddress/node/containerlogs/$containerId/$user" logDebug(s"Base URL for logs: $baseUrl") Some(Map( @@ -60,9 +59,7 @@ private[spark] object YarnContainerInfoHelper extends Logging { val yarnConf = new YarnConfiguration(conf) Some(Map( "HTTP_SCHEME" -> getYarnHttpScheme(yarnConf), - "NM_HOST" -> getNodeManagerHost, - "NM_PORT" -> getNodeManagerPort.toString, - "NM_HTTP_PORT" -> getNodeManagerHttpPort.toString, + "NM_HTTP_ADDRESS" -> getNodeManagerHttpAddress(container), "CLUSTER_ID" -> getClusterId(yarnConf).getOrElse(""), "CONTAINER_ID" -> ConverterUtils.toString(getContainerId(container)), "USER" -> Utils.getCurrentUserName(), @@ -98,9 +95,9 @@ private[spark] object YarnContainerInfoHelper extends Logging { if (yarnHttpPolicy == "HTTPS_ONLY") "https://" else "http://" } - def getNodeManagerHost: String = System.getenv(Environment.NM_HOST.name()) - - def getNodeManagerPort: Int = System.getenv(Environment.NM_PORT.name()).toInt - - def getNodeManagerHttpPort: Int = System.getenv(Environment.NM_HTTP_PORT.name()).toInt + def getNodeManagerHttpAddress(container: Option[Container]): String = container match { + case Some(c) => c.getNodeHttpAddress + case None => System.getenv(Environment.NM_HOST.name()) + ":" + + System.getenv(Environment.NM_HTTP_PORT.name()) + } } diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index d41b5e2278928..b3c5bbd263ed7 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -477,9 +477,7 @@ private object YarnClusterDriver extends Logging with Matchers { val driverAttributes = listener.driverAttributes.get val expectationAttributes = Map( "HTTP_SCHEME" -> YarnContainerInfoHelper.getYarnHttpScheme(yarnConf), - "NM_HOST" -> YarnContainerInfoHelper.getNodeManagerHost, - "NM_PORT" -> YarnContainerInfoHelper.getNodeManagerPort.toString, - "NM_HTTP_PORT" -> YarnContainerInfoHelper.getNodeManagerHttpPort.toString, + "NM_HTTP_ADDRESS" -> YarnContainerInfoHelper.getNodeManagerHttpAddress(container = None), "CLUSTER_ID" -> YarnContainerInfoHelper.getClusterId(yarnConf).getOrElse(""), "CONTAINER_ID" -> ConverterUtils.toString(containerId), "USER" -> user,